Merge branch 'master' into raman/fullstack_lsm_staging_coredump
diff --git a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index dde4443..e0ee1f0 100644
--- a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -37,6 +37,7 @@
 import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 
 public abstract class AbstractCompilerFactoryBuilder {
 
@@ -50,6 +51,7 @@
     protected IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
     protected IBinaryIntegerInspectorFactory binaryIntegerInspectorFactory;
     protected IPrinterFactoryProvider printerProvider;
+    protected IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider;
     protected IExpressionRuntimeProvider expressionRuntimeProvider;
     protected IExpressionTypeComputer expressionTypeComputer;
     protected INullableTypeComputer nullableTypeComputer;
@@ -111,6 +113,14 @@
     public IBinaryComparatorFactoryProvider getComparatorFactoryProvider() {
         return comparatorFactoryProvider;
     }
+    
+    public void setPredicateEvaluatorFactoryProvider(IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider) {
+        this.predEvaluatorFactoryProvider = predEvaluatorFactoryProvider;
+    }
+
+    public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactory() {
+        return predEvaluatorFactoryProvider;
+    }
 
     public void setBinaryBooleanInspectorFactory(IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory) {
         this.binaryBooleanInspectorFactory = binaryBooleanInspectorFactory;
diff --git a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index 2b24bd0..1c62307 100644
--- a/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/algebricks/algebricks-compiler/src/main/java/edu/uci/ics/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -91,7 +91,7 @@
                                 binaryBooleanInspectorFactory, binaryIntegerInspectorFactory, printerProvider,
                                 nullWriterFactory, normalizedKeyComputerFactoryProvider, expressionRuntimeProvider,
                                 expressionTypeComputer, nullableTypeComputer, oc, expressionEvalSizeComputer,
-                                partialAggregationTypeComputer, frameSize, clusterLocations);
+                                partialAggregationTypeComputer, predEvaluatorFactoryProvider, frameSize, clusterLocations);
                         PlanCompiler pc = new PlanCompiler(context);
                         return pc.compilePlan(plan, null, jobEventListenerFactory);
                     }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
index 2f53f9b..0c105d0 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/AggregateOperator.java
@@ -20,10 +20,11 @@
     // private ArrayList<AggregateFunctionCallExpression> expressions;
     // TODO type safe list of expressions
     private List<Mutable<ILogicalExpression>> mergeExpressions;
-    private LogicalVariable partitioningVariable;
+    private boolean global;
 
     public AggregateOperator(List<LogicalVariable> variables, List<Mutable<ILogicalExpression>> expressions) {
         super(variables, expressions);
+        global = true;
     }
 
     @Override
@@ -69,12 +70,12 @@
         return mergeExpressions;
     }
 
-    public void setPartitioningVariable(LogicalVariable partitioningVariable) {
-        this.partitioningVariable = partitioningVariable;
+    public void setGlobal(boolean global) {
+        this.global = global;
     }
 
-    public LogicalVariable getPartitioningVariable() {
-        return partitioningVariable;
+    public boolean isGlobal() {
+        return global;
     }
 
     @Override
@@ -90,4 +91,5 @@
         }
         return env;
     }
+
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
index b1da831..fed1a15 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
@@ -28,7 +28,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;

 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;

 import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;

-import edu.uci.ics.hyracks.algebricks.core.algebra.typing.NonPropagatingTypeEnvironment;

 import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;

 import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;

 

@@ -50,15 +49,12 @@
 

     @Override

     public void recomputeSchema() {

-        if (schema == null) {

-            schema = new ArrayList<LogicalVariable>();

-        }

-        schema.clear();

-        for (Mutable<ILogicalExpression> eRef : expressions) {

-            ILogicalExpression e = eRef.getValue();

-            if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

-                VariableReferenceExpression v = (VariableReferenceExpression) e;

-                schema.add(v.getVariableReference());

+        schema = new ArrayList<LogicalVariable>();

+        schema.addAll(this.getDistinctByVarList());

+        List<LogicalVariable> inputSchema = inputs.get(0).getValue().getSchema();

+        for (LogicalVariable var : inputSchema) {

+            if (!schema.contains(var)) {

+                schema.add(var);

             }

         }

     }

@@ -69,12 +65,16 @@
             @Override

             public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)

                     throws AlgebricksException {

-                for (Mutable<ILogicalExpression> eRef : expressions) {

-                    ILogicalExpression e = eRef.getValue();

-                    if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

-                        VariableReferenceExpression v = (VariableReferenceExpression) e;

-                        target.addVariable(v.getVariableReference());

-                    }

+                /** make sure distinct key vars laid-out first */

+                for (LogicalVariable keyVar : getDistinctByVarList()) {

+                    target.addVariable(keyVar);

+                }

+                /** add other source vars */

+                for (IOperatorSchema srcSchema : sources) {

+                    for (LogicalVariable srcVar : srcSchema)

+                        if (target.findVariable(srcVar) < 0) {

+                            target.addVariable(srcVar);

+                        }

                 }

             }

         };

@@ -128,16 +128,7 @@
 

     @Override

     public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {

-        IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMetadataProvider());

-        IVariableTypeEnvironment childEnv = ctx.getOutputTypeEnvironment(inputs.get(0).getValue());

-        for (Mutable<ILogicalExpression> exprRef : expressions) {

-            ILogicalExpression expr = exprRef.getValue();

-            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

-                VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr;

-                env.setVarType(varRefExpr.getVariableReference(), childEnv.getType(expr));

-            }

-        }

-        return env;

+        return createPropagatingAllInputsTypeEnvironment(ctx);

     }

 

 }

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 0de9652..5225ac7 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -14,7 +14,9 @@
  */

 package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;

 

+import java.util.ArrayList;

 import java.util.Collection;

+import java.util.List;

 

 import org.apache.commons.lang3.mutable.Mutable;

 

@@ -85,11 +87,17 @@
 

     @Override

     public Void visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {

-        for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {

-            ILogicalExpression expr = exprRef.getValue();

-            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {

-                VariableReferenceExpression varRefExpr = (VariableReferenceExpression) expr;

-                schemaVariables.add(varRefExpr.getVariableReference());

+        List<LogicalVariable> allLiveVars = new ArrayList<LogicalVariable>();

+        for (Mutable<ILogicalOperator> c : op.getInputs()) {

+            VariableUtilities.getLiveVariables(c.getValue(), allLiveVars);

+        }

+        VariableUtilities.getProducedVariables(op, allLiveVars);

+        /** put distinct vars first */

+        schemaVariables.addAll(op.getDistinctByVarList());

+        /** then other live vars */

+        for (LogicalVariable var : allLiveVars) {

+            if (!schemaVariables.contains(var)) {

+                schemaVariables.add(var);

             }

         }

         return null;

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index a620e54..42f7e20 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -77,9 +77,6 @@
         for (Mutable<ILogicalExpression> exprRef : op.getExpressions()) {

             exprRef.getValue().getUsedVariables(usedVariables);

         }

-        if (op.getPartitioningVariable() != null) {

-            usedVariables.add(op.getPartitioningVariable());

-        }

         return null;

     }

 

diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
index 81dc2c2..f47c2ec 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
@@ -16,12 +16,10 @@
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.ListSet;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -31,15 +29,14 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
@@ -58,25 +55,29 @@
 
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        IPhysicalPropertiesVector childProps = op2.getDeliveredPhysicalProperties();
-        deliveredProperties = new StructuralPropertiesVector(childProps.getPartitioningProperty(),
-                new ArrayList<ILocalStructuralProperty>(0));
+        AggregateOperator aggOp = (AggregateOperator) op;
+        ILogicalOperator op2 = op.getInputs().get(0).getValue();
+        if (aggOp.getExecutionMode() != AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) {
+            deliveredProperties = new StructuralPropertiesVector(op2.getDeliveredPhysicalProperties()
+                    .getPartitioningProperty(), new ArrayList<ILocalStructuralProperty>());
+        } else {
+            deliveredProperties = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED,
+                    new ArrayList<ILocalStructuralProperty>());
+        }
     }
 
     @Override
     public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
             IPhysicalPropertiesVector reqdByParent) {
         AggregateOperator aggOp = (AggregateOperator) op;
-        if (aggOp.getExecutionMode() == ExecutionMode.PARTITIONED && aggOp.getPartitioningVariable() != null) {
-            StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
-            Set<LogicalVariable> partitioningVariables = new ListSet<LogicalVariable>();
-            partitioningVariables.add(aggOp.getPartitioningVariable());
-            pv[0] = new StructuralPropertiesVector(new UnorderedPartitionedProperty(partitioningVariables, null), null);
+        StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+        if (aggOp.isGlobal() && aggOp.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) {
+            pv[0] = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED, null);
             return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
         } else {
             return emptyUnaryRequirements();
         }
+
     }
 
     @Override
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
index 302d4d2..9a96319 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
@@ -100,10 +100,9 @@
                 context, columns);
 
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getResultHandleRuntime(
-                resultOp.getDataSink(), columns, pf, inputDesc, false, spec);
+                resultOp.getDataSink(), columns, pf, inputDesc, true, spec);
 
         builder.contributeHyracksOperator(resultOp, runtimeAndConstraints.first);
-        builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
         ILogicalOperator src = resultOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, resultOp, 0);
     }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index 6da42b4..d745abd 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -40,6 +40,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -106,6 +108,10 @@
             Object t = env.getVarType(v);
             comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, true);
         }
+        
+        IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider = context.getPredicateEvaluatorFactoryProvider();
+        IPredicateEvaluatorFactory predEvaluatorFactory = ( predEvaluatorFactoryProvider == null ? null : predEvaluatorFactoryProvider.getPredicateEvaluatorFactory(keysLeft, keysRight));
+        
         RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
                 propagatedSchema, context);
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
@@ -125,7 +131,7 @@
                     case INNER: {
                         opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
                                 maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
-                                hashFunFactories, comparatorFactories, recDescriptor);
+                                hashFunFactories, comparatorFactories, recDescriptor, predEvaluatorFactory);
                         break;
                     }
                     case LEFT_OUTER: {
@@ -135,7 +141,7 @@
                         }
                         opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
                                 maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
-                                hashFunFactories, comparatorFactories, recDescriptor, true, nullWriterFactories);
+                                hashFunFactories, comparatorFactories, recDescriptor, predEvaluatorFactory, true, nullWriterFactories);
                         break;
                     }
                     default: {
@@ -153,7 +159,7 @@
                                 maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
                                 comparatorFactories, recDescriptor, new JoinMultiComparatorFactory(comparatorFactories,
                                         keysLeft, keysRight), new JoinMultiComparatorFactory(comparatorFactories,
-                                        keysRight, keysLeft));
+                                        keysRight, keysLeft), predEvaluatorFactory);
                         break;
                     }
                     case LEFT_OUTER: {
@@ -165,7 +171,7 @@
                                 maxInputBuildSizeInFrames, getFudgeFactor(), keysLeft, keysRight, hashFunFamilies,
                                 comparatorFactories, recDescriptor, new JoinMultiComparatorFactory(comparatorFactories,
                                         keysLeft, keysRight), new JoinMultiComparatorFactory(comparatorFactories,
-                                        keysRight, keysLeft), true, nullWriterFactories);
+                                        keysRight, keysLeft), predEvaluatorFactory, true, nullWriterFactories);
                         break;
                     }
                     default: {
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index b09c194..2c59151 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -37,6 +37,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
@@ -86,6 +88,10 @@
             Object t = env.getVarType(v);
             comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, true);
         }
+        
+        IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider = context.getPredicateEvaluatorFactoryProvider();
+        IPredicateEvaluatorFactory predEvaluatorFactory = ( predEvaluatorFactoryProvider == null ? null : predEvaluatorFactoryProvider.getPredicateEvaluatorFactory(keysLeft, keysRight));
+        
         RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
                 propagatedSchema, context);
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
@@ -94,7 +100,7 @@
         switch (kind) {
             case INNER: {
                 opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, hashFunFactories,
-                        comparatorFactories, recDescriptor, tableSize);
+                        comparatorFactories, recDescriptor, tableSize, predEvaluatorFactory);
                 break;
             }
             case LEFT_OUTER: {
@@ -103,7 +109,7 @@
                     nullWriterFactories[j] = context.getNullWriterFactory();
                 }
                 opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, hashFunFactories,
-                        comparatorFactories, recDescriptor, true, nullWriterFactories, tableSize);
+                        comparatorFactories, predEvaluatorFactory, recDescriptor, true, nullWriterFactories, tableSize);
                 break;
             }
             default: {
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
index e000b85..5fd6ef4 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical;
 
 import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -22,13 +21,8 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -45,13 +39,13 @@
     private final IDataSourceIndex<?, ?> dataSourceIndex;
 
     public IndexInsertDeletePOperator(List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
-    		Mutable<ILogicalExpression> filterExpr, IDataSourceIndex<?, ?> dataSourceIndex) {
+            Mutable<ILogicalExpression> filterExpr, IDataSourceIndex<?, ?> dataSourceIndex) {
         this.primaryKeys = primaryKeys;
         this.secondaryKeys = secondaryKeys;
         if (filterExpr != null) {
-        	this.filterExpr = filterExpr.getValue();
+            this.filterExpr = filterExpr.getValue();
         } else {
-        	this.filterExpr = null;
+            this.filterExpr = null;
         }
         this.dataSourceIndex = dataSourceIndex;
     }
@@ -64,7 +58,7 @@
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();        
+        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
     }
 
     @Override
@@ -73,15 +67,11 @@
         List<LogicalVariable> scanVariables = new ArrayList<LogicalVariable>();
         scanVariables.addAll(primaryKeys);
         scanVariables.add(new LogicalVariable(-1));
-        IPartitioningProperty pp = dataSourceIndex.getDataSource().getPropertiesProvider()
-                .computePropertiesVector(scanVariables).getPartitioningProperty();
-        List<ILocalStructuralProperty> orderProps = new LinkedList<ILocalStructuralProperty>();
-        for (LogicalVariable k : secondaryKeys) {
-            orderProps.add(new LocalOrderProperty(new OrderColumn(k, OrderKind.ASC)));
-        }
-        StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp,
-                orderProps) };
-        return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+        IPhysicalPropertiesVector r = dataSourceIndex.getDataSource().getPropertiesProvider()
+                .computePropertiesVector(scanVariables);
+        IPhysicalPropertiesVector[] requirements = new IPhysicalPropertiesVector[1];
+        requirements[0] = r;
+        return new PhysicalRequirements(requirements, IPartitioningRequirementsCoordinator.NO_COORDINATION);
     }
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
@@ -93,8 +83,8 @@
         IMetadataProvider mp = context.getMetadataProvider();
 
         JobSpecification spec = builder.getJobSpec();
-        RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0],
-                context);
+        RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
+                context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
 
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(insertDeleteOp);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
index 29af97c..b5656cd 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
@@ -105,14 +105,24 @@
                 fdColumns[j++] = inputSchemas[0].findVariable(v);
             }
         }
+        int[] keysAndDecs = new int[keys.length + fdColumns.length];
+        for (int i = 0; i < keys.length; i++) {
+            keysAndDecs[i] = keys[i];
+        }
+        for (int i = 0; i < fdColumns.length; i++) {
+            keysAndDecs[i + keys.length] = fdColumns[i];
+        }
+
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
                 columnList, context.getTypeEnvironment(op), context);
         IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[] {};
         IAggregatorDescriptorFactory aggregatorFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(
-                aggFactories, keys, fdColumns);
+                aggFactories, keysAndDecs);
 
-        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
-        PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
+        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema,
+                context);
+        /** make fd columns part of the key but the comparator only compares the distinct key columns */
+        PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keysAndDecs,
                 comparatorFactories, aggregatorFactory, recordDescriptor);
 
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
index 11e24d7..531b300 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
@@ -37,10 +38,8 @@
 
 public class StreamLimitPOperator extends AbstractPhysicalOperator {
 
-    private boolean global;
+    public StreamLimitPOperator() {
 
-    public StreamLimitPOperator(boolean global) {
-        this.global = global;
     }
 
     @Override
@@ -55,14 +54,22 @@
 
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+        AbstractLogicalOperator limitOp = (AbstractLogicalOperator) op;
         ILogicalOperator op2 = op.getInputs().get(0).getValue();
-        deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+        if (limitOp.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) {
+            //partitioning property: unpartitioned;  local property: whatever from the child
+            deliveredProperties = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED, op2
+                    .getDeliveredPhysicalProperties().getLocalProperties());
+        } else {
+            deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+        }
     }
 
     @Override
     public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
             IPhysicalPropertiesVector reqdByParent) {
-        if (global) {
+        AbstractLogicalOperator limitOp = (AbstractLogicalOperator) op;
+        if (limitOp.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) {
             StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
             pv[0] = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED, null);
             return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
@@ -83,7 +90,8 @@
         ILogicalExpression offsetExpr = limit.getOffset().getValue();
         IScalarEvaluatorFactory offsetFact = (offsetExpr == null) ? null : expressionRuntimeProvider
                 .createEvaluatorFactory(offsetExpr, env, inputSchemas, context);
-        RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+        RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema,
+                context);
         StreamLimitRuntimeFactory runtime = new StreamLimitRuntimeFactory(maxObjectsFact, offsetFact, null,
                 context.getBinaryIntegerInspectorFactory());
         builder.contributeMicroOperator(limit, runtime, recDesc);
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ResultSetDomain.java
similarity index 68%
rename from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
rename to algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ResultSetDomain.java
index 8f5ed64..ca15346 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/properties/ResultSetDomain.java
@@ -3,19 +3,25 @@
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- *
+ * 
  *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataset;
+package edu.uci.ics.hyracks.algebricks.core.algebra.properties;
 
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+public class ResultSetDomain implements INodeDomain {
+    @Override
+    public boolean sameAs(INodeDomain domain) {
+        return true;
+    }
 
-public interface IDatasetPartitionReader {
-    public void writeTo(IFrameWriter writer);
+    @Override
+    public Integer cardinality() {
+        return 0;
+    }
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index 47a979c..35b36dc 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
@@ -96,14 +97,21 @@
                 break;
             }
             default: {
+                boolean forceUnpartitioned = false;
                 if (op.getOperatorTag() == LogicalOperatorTag.LIMIT) {
                     LimitOperator opLim = (LimitOperator) op;
                     if (opLim.isTopmostLimitOp()) {
-                        if (opLim.getExecutionMode() != AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) {
-                            opLim.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
-                            change = true;
-                        }
-                        break;
+                        opLim.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+                        change = true;
+                        forceUnpartitioned = true;
+                    }
+                }
+                if (op.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
+                    AggregateOperator aggOp = (AggregateOperator) op;
+                    if (aggOp.isGlobal()) {
+                        op.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+                        change = true;
+                        forceUnpartitioned = true;
                     }
                 }
 
@@ -112,6 +120,8 @@
                     AbstractLogicalOperator inputOp = (AbstractLogicalOperator) i.getValue();
                     switch (inputOp.getExecutionMode()) {
                         case PARTITIONED: {
+                            if (forceUnpartitioned)
+                                break;
                             op.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
                             change = true;
                             exit = true;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 365d1a5..245e5e1 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -41,6 +41,8 @@
 import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
 
 public class JobGenContext {
 	private final IOperatorSchema outerFlowSchema;
@@ -61,6 +63,7 @@
 	private final IExpressionTypeComputer expressionTypeComputer;
 	private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
 	private final IPartialAggregationTypeComputer partialAggregationTypeComputer;
+	private final IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider;
 	private final int frameSize;
 	private AlgebricksPartitionConstraint clusterLocations;
 	private int varCounter;
@@ -86,7 +89,7 @@
 			ITypingContext typingContext,
 			IExpressionEvalSizeComputer expressionEvalSizeComputer,
 			IPartialAggregationTypeComputer partialAggregationTypeComputer,
-			int frameSize, AlgebricksPartitionConstraint clusterLocations) {
+			IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider, int frameSize, AlgebricksPartitionConstraint clusterLocations) {
 		this.outerFlowSchema = outerFlowSchema;
 		this.metadataProvider = metadataProvider;
 		this.appContext = appContext;
@@ -106,6 +109,7 @@
 		this.typingContext = typingContext;
 		this.expressionEvalSizeComputer = expressionEvalSizeComputer;
 		this.partialAggregationTypeComputer = partialAggregationTypeComputer;
+		this.predEvaluatorFactoryProvider = predEvaluatorFactoryProvider;
 		this.frameSize = frameSize;
 		this.varCounter = 0;
 	}
@@ -157,6 +161,10 @@
 	public IPrinterFactoryProvider getPrinterFactoryProvider() {
 		return printerFactoryProvider;
 	}
+	
+	public IPredicateEvaluatorFactoryProvider getPredicateEvaluatorFactoryProvider(){
+		return predEvaluatorFactoryProvider;
+	}
 
 	public IExpressionRuntimeProvider getExpressionRuntimeProvider() {
 		return expressionRuntimeProvider;
diff --git a/algebricks/algebricks-examples/piglet-example/pom.xml b/algebricks/algebricks-examples/piglet-example/pom.xml
index 4b101d7..254d016 100644
--- a/algebricks/algebricks-examples/piglet-example/pom.xml
+++ b/algebricks/algebricks-examples/piglet-example/pom.xml
@@ -48,6 +48,24 @@
 					</includes>
 				</configuration>
 			</plugin>
+                        <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>${project.build.directory}/generated-sources/javacc/</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
 		</plugins>
 		<pluginManagement>
 			<plugins>
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
index 08271c1..013ddda 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/AbstractIntroduceCombinerRule.java
@@ -16,12 +16,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
@@ -110,16 +108,10 @@
             } else {
                 // The local aggregate operator is fed by the input of the original aggregate operator.
                 pushedAgg.getInputs().add(new MutableObject<ILogicalOperator>(initAgg.getInputs().get(0).getValue()));
-                // Set the partitioning variable in the local agg to ensure it is not projected away.
-                context.computeAndSetTypeEnvironmentForOperator(pushedAgg);
-                LogicalVariable trueVar = context.newVar();
                 // Reintroduce assign op for the global agg partitioning var.
-                AssignOperator trueAssignOp = new AssignOperator(trueVar, new MutableObject<ILogicalExpression>(
-                        ConstantExpression.TRUE));
-                trueAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(pushedAgg));
-                context.computeAndSetTypeEnvironmentForOperator(trueAssignOp);
-                initAgg.setPartitioningVariable(trueVar);
-                initAgg.getInputs().get(0).setValue(trueAssignOp);
+                initAgg.getInputs().get(0).setValue(pushedAgg);
+                pushedAgg.setGlobal(false);
+                context.computeAndSetTypeEnvironmentForOperator(pushedAgg);
             }
             return new Pair<Boolean, Mutable<ILogicalOperator>>(true, new MutableObject<ILogicalOperator>(pushedAgg));
         } else {
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
index fa5000e..06172d9 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ComplexUnnestToProductRule.java
@@ -55,6 +55,11 @@
             return false;
         }
 
+        //stop rewriting if the operators originates from a nested tuple source
+        if (insideSubplan(opRef)) {
+            return false;
+        }
+
         // We may pull selects above the join we create in order to eliminate possible dependencies between
         // the outer and inner input plans of the join.
         List<ILogicalOperator> topSelects = new ArrayList<ILogicalOperator>();
@@ -286,4 +291,24 @@
         return findPlanPartition((AbstractLogicalOperator) op.getInputs().get(0).getValue(), innerUsedVars,
                 outerUsedVars, innerOps, outerOps, topSelects, belowSecondUnnest);
     }
+
+    /**
+     * check whether the operator is inside a sub-plan
+     * 
+     * @param nestedRootRef
+     * @return true-if it is; false otherwise.
+     */
+    private boolean insideSubplan(Mutable<ILogicalOperator> nestedRootRef) {
+        AbstractLogicalOperator nestedRoot = (AbstractLogicalOperator) nestedRootRef.getValue();
+        if (nestedRoot.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+            return true;
+        }
+        List<Mutable<ILogicalOperator>> inputs = nestedRoot.getInputs();
+        for (Mutable<ILogicalOperator> input : inputs) {
+            if (insideSubplan(input)) {
+                return true;
+            }
+        }
+        return false;
+    }
 }
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
index 3260ca0..00670fa 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java
@@ -45,12 +45,18 @@
 public class EnforceOrderByAfterSubplan implements IAlgebraicRewriteRule {
     /** a set of order-breaking operators */
     private final Set<LogicalOperatorTag> orderBreakingOps = new HashSet<LogicalOperatorTag>();
+    /** a set of order-sensitive operators */
+    private final Set<LogicalOperatorTag> orderSensitiveOps = new HashSet<LogicalOperatorTag>();
 
     public EnforceOrderByAfterSubplan() {
         /** add operators that break the ordering */
         orderBreakingOps.add(LogicalOperatorTag.INNERJOIN);
         orderBreakingOps.add(LogicalOperatorTag.LEFTOUTERJOIN);
         orderBreakingOps.add(LogicalOperatorTag.UNIONALL);
+        orderBreakingOps.add(LogicalOperatorTag.AGGREGATE);
+
+        /** add operators that are sensitive to the ordering */
+        orderSensitiveOps.add(LogicalOperatorTag.LIMIT);
     }
 
     @Override
@@ -89,19 +95,25 @@
              * duplicate them on-top-of the subplan operator
              */
             boolean foundTarget = true;
-            AbstractLogicalOperator child = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+            boolean orderSensitive = false;
+            Mutable<ILogicalOperator> childRef = op.getInputs().get(0);
+            AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
             while (child.getOperatorTag() != LogicalOperatorTag.ORDER) {
                 context.addToDontApplySet(this, child);
                 if (orderBreakingOps.contains(child.getOperatorTag())) {
                     foundTarget = false;
                     break;
                 }
+                if (orderSensitiveOps.contains(child.getOperatorTag())) {
+                    orderSensitive = true;
+                }
                 List<Mutable<ILogicalOperator>> childInputs = child.getInputs();
                 if (childInputs == null || childInputs.size() > 2 || childInputs.size() < 1) {
                     foundTarget = false;
                     break;
                 } else {
-                    child = (AbstractLogicalOperator) childInputs.get(0).getValue();
+                    childRef = childInputs.get(0);
+                    child = (AbstractLogicalOperator) childRef.getValue();
                 }
             }
             /** the target order-by operator has not been found. */
@@ -109,7 +121,7 @@
                 return false;
             }
 
-            /** duplicate the order-by operator and insert on-top-of the subplan operator */
+            /** copy the original order-by operator and insert on-top-of the subplan operator */
             context.addToDontApplySet(this, child);
             OrderOperator sourceOrderOp = (OrderOperator) child;
             List<Pair<IOrder, Mutable<ILogicalExpression>>> orderExprs = deepCopyOrderAndExpression(sourceOrderOp
@@ -119,6 +131,11 @@
             inputs.set(i, new MutableObject<ILogicalOperator>(newOrderOp));
             newOrderOp.getInputs().add(inputOpRef);
             context.computeAndSetTypeEnvironmentForOperator(newOrderOp);
+
+            if (!orderSensitive) {
+                /** remove the original order-by */
+                childRef.setValue(sourceOrderOp.getInputs().get(0).getValue());
+            }
             changed = true;
         }
         return changed;
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 f017e0f..9becf6e 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
@@ -48,57 +48,46 @@
 
 /**
  * Factors out common sub-expressions by assigning them to a variables, and replacing the common sub-expressions with references to those variables.
- *
  * Preconditions/Assumptions:
  * Assumes no projects are in the plan. This rule ignores variable reference expressions and constants (other rules deal with those separately).
- * 
  * Postconditions/Examples:
  * Plan with extracted sub-expressions. Generates one assign operator per extracted expression.
- * 
  * Example 1 - Simple Arithmetic Example (simplified)
- * 
  * Before plan:
  * assign [$$1] <- [5 + 6 - 10]
- *   assign [$$0] <- [5 + 6 + 30]
- * 
+ * assign [$$0] <- [5 + 6 + 30]
  * After plan:
  * assign [$$1] <- [$$5 - 10]
- *   assign [$$0] <- [$$5 + 30]
- *     assign [$$5] <- [5 + 6]
- * 
+ * assign [$$0] <- [$$5 + 30]
+ * assign [$$5] <- [5 + 6]
  * Example 2 - Cleaning up 'Distinct By' (simplified)
- * 
  * Before plan: (notice how $$0 is not live after the distinct)
  * assign [$$3] <- [field-access($$0, 1)]
- *   distinct ([%0->$$5])
- *     assign [$$5] <- [field-access($$0, 1)]
- *       unnest $$0 <- [scan-dataset]
- * 
+ * distinct ([%0->$$5])
+ * assign [$$5] <- [field-access($$0, 1)]
+ * unnest $$0 <- [scan-dataset]
  * After plan: (notice how the issue of $$0 is fixed)
  * assign [$$3] <- [$$5]
- *   distinct ([$$5])
- *     assign [$$5] <- [field-access($$0, 1)]
- *       unnest $$0 <- [scan-dataset]
- * 
+ * distinct ([$$5])
+ * assign [$$5] <- [field-access($$0, 1)]
+ * unnest $$0 <- [scan-dataset]
  * Example 3 - Pulling Common Expressions Above Joins (simplified)
- * 
  * Before plan:
  * assign [$$9] <- funcZ(funcY($$8))
- *   join (funcX(funcY($$8)))
- * 
+ * join (funcX(funcY($$8)))
  * After plan:
  * assign [$$9] <- funcZ($$10))
- *   select (funcX($$10))
- *     assign [$$10] <- [funcY($$8)]
- *       join (TRUE)
+ * select (funcX($$10))
+ * assign [$$10] <- [funcY($$8)]
+ * join (TRUE)
  */
 public class ExtractCommonExpressionsRule implements IAlgebraicRewriteRule {
 
     private final List<ILogicalExpression> originalAssignExprs = new ArrayList<ILogicalExpression>();
-    
+
     private final CommonExpressionSubstitutionVisitor substVisitor = new CommonExpressionSubstitutionVisitor();
     private final Map<ILogicalExpression, ExprEquivalenceClass> exprEqClassMap = new HashMap<ILogicalExpression, ExprEquivalenceClass>();
-    
+
     // Set of operators for which common subexpression elimination should not be performed.
     private static final Set<LogicalOperatorTag> ignoreOps = new HashSet<LogicalOperatorTag>();
     static {
@@ -109,9 +98,10 @@
         ignoreOps.add(LogicalOperatorTag.AGGREGATE);
         ignoreOps.add(LogicalOperatorTag.RUNNINGAGGREGATE);
     }
-    
+
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         return false;
     }
 
@@ -126,7 +116,8 @@
         return modified;
     }
 
-    private void updateEquivalenceClassMap(LogicalVariable lhs, Mutable<ILogicalExpression> rhsExprRef, ILogicalExpression rhsExpr, ILogicalOperator op) {
+    private void updateEquivalenceClassMap(LogicalVariable lhs, Mutable<ILogicalExpression> rhsExprRef,
+            ILogicalExpression rhsExpr, ILogicalOperator op) {
         ExprEquivalenceClass exprEqClass = exprEqClassMap.get(rhsExpr);
         if (exprEqClass == null) {
             exprEqClass = new ExprEquivalenceClass(op, rhsExprRef);
@@ -141,7 +132,7 @@
         if (context.checkIfInDontApplySet(this, opRef.getValue())) {
             return false;
         }
-        
+
         boolean modified = false;
         // Recurse into children.
         for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
@@ -149,7 +140,7 @@
                 modified = true;
             }
         }
-        
+
         // TODO: Deal with replicate properly. Currently, we just clear the expr equivalence map, since we want to avoid incorrect expression replacement
         // (the resulting new variables should be assigned live below a replicate).
         if (op.getOperatorTag() == LogicalOperatorTag.REPLICATE) {
@@ -160,7 +151,7 @@
         if (ignoreOps.contains(op.getOperatorTag())) {
             return modified;
         }
-        
+
         // Remember a copy of the original assign expressions, so we can add them to the equivalence class map
         // after replacing expressions within the assign operator itself.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
@@ -173,13 +164,13 @@
                 originalAssignExprs.add(expr.cloneExpression());
             }
         }
-        
+
         // Perform common subexpression elimination.
         substVisitor.setOperator(op);
         if (op.acceptExpressionTransform(substVisitor)) {
             modified = true;
         }
-        
+
         // Update equivalence class map.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AssignOperator assignOp = (AssignOperator) op;
@@ -194,7 +185,7 @@
                 // Update equivalence class map.
                 LogicalVariable lhs = assignOp.getVariables().get(i);
                 updateEquivalenceClassMap(lhs, exprRef, exprRef.getValue(), op);
-                
+
                 // Update equivalence class map with original assign expression.
                 updateEquivalenceClassMap(lhs, exprRef, originalAssignExprs.get(i), op);
             }
@@ -225,35 +216,30 @@
     }
 
     private class CommonExpressionSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
-                
-        private final Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
-        private final List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+
         private IOptimizationContext context;
-        private ILogicalOperator op;        
-        
+        private ILogicalOperator op;
+
         public void setContext(IOptimizationContext context) {
             this.context = context;
         }
-        
+
         public void setOperator(ILogicalOperator op) throws AlgebricksException {
             this.op = op;
-            liveVars.clear();
-            usedVars.clear();
         }
-        
+
         @Override
         public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
-            if (liveVars.isEmpty() && usedVars.isEmpty()) {
-                VariableUtilities.getLiveVariables(op, liveVars);
-                VariableUtilities.getUsedVariables(op, usedVars);
-            }
-            
             AbstractLogicalExpression expr = (AbstractLogicalExpression) exprRef.getValue();
             boolean modified = false;
             ExprEquivalenceClass exprEqClass = exprEqClassMap.get(expr);
             if (exprEqClass != null) {
                 // Replace common subexpression with existing variable. 
                 if (exprEqClass.variableIsSet()) {
+                    Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
+                    List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+                    VariableUtilities.getLiveVariables(op, liveVars);
+                    VariableUtilities.getUsedVariables(op, usedVars);
                     // Check if the replacing variable is live at this op.
                     // However, if the op is already using variables that are not live, then a replacement may enable fixing the plan.
                     // This behavior is necessary to, e.g., properly deal with distinct by.
@@ -266,9 +252,15 @@
                     }
                 } else {
                     if (assignCommonExpression(exprEqClass, expr)) {
-                        exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
-                        // Do not descend into children since this expr has been completely replaced.
-                        return true;
+                        //re-obtain the live vars after rewriting in the method called in the if condition
+                        Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
+                        VariableUtilities.getLiveVariables(op, liveVars);
+                        //rewrite only when the variable is live
+                        if (liveVars.contains(exprEqClass.getVariable())) {
+                            exprRef.setValue(new VariableReferenceExpression(exprEqClass.getVariable()));
+                            // Do not descend into children since this expr has been completely replaced.
+                            return true;
+                        }
                     }
                 }
             } else {
@@ -278,7 +270,7 @@
                     exprEqClassMap.put(expr, exprEqClass);
                 }
             }
-            
+
             // Descend into function arguments.
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
@@ -290,17 +282,19 @@
             }
             return modified;
         }
-        
-        private boolean assignCommonExpression(ExprEquivalenceClass exprEqClass, ILogicalExpression expr) throws AlgebricksException {
+
+        private boolean assignCommonExpression(ExprEquivalenceClass exprEqClass, ILogicalExpression expr)
+                throws AlgebricksException {
             AbstractLogicalOperator firstOp = (AbstractLogicalOperator) exprEqClass.getFirstOperator();
             Mutable<ILogicalExpression> firstExprRef = exprEqClass.getFirstExpression();
-            if (firstOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN || firstOp.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+            if (firstOp.getOperatorTag() == LogicalOperatorTag.INNERJOIN
+                    || firstOp.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
                 // Do not extract common expressions from within the same join operator.
                 if (firstOp == op) {
                     return false;
                 }
                 AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) firstOp;
-                Mutable<ILogicalExpression> joinCond = joinOp.getCondition();                
+                Mutable<ILogicalExpression> joinCond = joinOp.getCondition();
                 ILogicalExpression enclosingExpr = getEnclosingExpression(joinCond, firstExprRef.getValue());
                 if (enclosingExpr == null) {
                     // No viable enclosing expression that we can pull out from the join.
@@ -312,12 +306,13 @@
                 op.getInputs().get(0).setValue(selectOp);
                 // Set firstOp to be the select below op, since we want to assign the common subexpr there.
                 firstOp = (AbstractLogicalOperator) selectOp;
-            } else if (firstOp.getInputs().size() > 1) { 
+            } else if (firstOp.getInputs().size() > 1) {
                 // Bail for any non-join operator with multiple inputs.
                 return false;
-            }                        
+            }
             LogicalVariable newVar = context.newVar();
-            AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(firstExprRef.getValue().cloneExpression()));            
+            AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(firstExprRef
+                    .getValue().cloneExpression()));
             // Place assign below firstOp.
             newAssign.getInputs().add(new MutableObject<ILogicalOperator>(firstOp.getInputs().get(0).getValue()));
             newAssign.setExecutionMode(firstOp.getExecutionMode());
@@ -330,7 +325,8 @@
             return true;
         }
 
-        private ILogicalExpression getEnclosingExpression(Mutable<ILogicalExpression> conditionExprRef, ILogicalExpression commonSubExpr) {
+        private ILogicalExpression getEnclosingExpression(Mutable<ILogicalExpression> conditionExprRef,
+                ILogicalExpression commonSubExpr) {
             ILogicalExpression conditionExpr = conditionExprRef.getValue();
             if (conditionExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
                 return null;
@@ -371,7 +367,7 @@
             return enclosingBoolExpr;
         }
     }
-    
+
     private boolean containsExpr(ILogicalExpression expr, ILogicalExpression searchExpr) {
         if (expr == searchExpr) {
             return true;
@@ -387,7 +383,7 @@
         }
         return false;
     }
-    
+
     private boolean isEqJoinCondition(ILogicalExpression expr) {
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
         if (funcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.EQ)) {
@@ -400,38 +396,38 @@
         }
         return false;
     }
-    
+
     private final class ExprEquivalenceClass {
         // First operator in which expression is used.
         private final ILogicalOperator firstOp;
-        
+
         // Reference to expression in first op.
         private final Mutable<ILogicalExpression> firstExprRef;
-        
+
         // Variable that this expression has been assigned to.
         private LogicalVariable var;
-        
+
         public ExprEquivalenceClass(ILogicalOperator firstOp, Mutable<ILogicalExpression> firstExprRef) {
             this.firstOp = firstOp;
             this.firstExprRef = firstExprRef;
         }
-        
+
         public ILogicalOperator getFirstOperator() {
             return firstOp;
         }
-        
+
         public Mutable<ILogicalExpression> getFirstExpression() {
             return firstExprRef;
         }
-        
+
         public void setVariable(LogicalVariable var) {
             this.var = var;
         }
-        
+
         public LogicalVariable getVariable() {
             return var;
         }
-        
+
         public boolean variableIsSet() {
             return var != null;
         }
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceAggregateCombinerRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceAggregateCombinerRule.java
index c3d935c..c75db57 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceAggregateCombinerRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroduceAggregateCombinerRule.java
@@ -29,7 +29,7 @@
             return false;
         }
         AggregateOperator aggOp = (AggregateOperator) op;
-        if (aggOp.getExecutionMode() != ExecutionMode.PARTITIONED || aggOp.getPartitioningVariable() == null) {
+        if (!aggOp.isGlobal() || aggOp.getExecutionMode() == ExecutionMode.LOCAL) {
             return false;
         }
         Map<AggregateFunctionCallExpression, SimilarAggregatesInfo> toReplaceMap = new HashMap<AggregateFunctionCallExpression, SimilarAggregatesInfo>();
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 18c483e..489167f 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
@@ -143,8 +143,10 @@
 
         Set<LogicalVariable> pkVars = computeGbyVars(outerNts, free, context);
         if (pkVars == null || pkVars.size() < 1) {
-            // could not group only by primary keys
-            return false;
+            // there is no non-trivial primary key, group-by keys are all live variables
+            ILogicalOperator subplanInput = subplan.getInputs().get(0).getValue();
+            pkVars = new HashSet<LogicalVariable>();
+            VariableUtilities.getLiveVariables(subplanInput, pkVars);
         }
         AlgebricksConfig.ALGEBRICKS_LOGGER.fine("Found FD for introducing group-by: " + pkVars);
 
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java
index acfdbd3..e5f60bc 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushLimitDownRule.java
@@ -106,7 +106,7 @@
                     opLim.getMaxObjects(), opLim.getOffset());
             clone2 = new LimitOperator(maxPlusOffset, false);
         }
-        clone2.setPhysicalOperator(new StreamLimitPOperator(false));
+        clone2.setPhysicalOperator(new StreamLimitPOperator());
         clone2.getInputs().add(new MutableObject<ILogicalOperator>(op2));
         clone2.setExecutionMode(op2.getExecutionMode());
         clone2.recomputeSchema();
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index f1b9a7b..9c8ad46 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -21,7 +21,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -172,9 +171,7 @@
                     break;
                 }
                 case LIMIT: {
-                    LimitOperator opLim = (LimitOperator) op;
-                    op.setPhysicalOperator(new StreamLimitPOperator(opLim.isTopmostLimitOp()
-                            && opLim.getExecutionMode() == ExecutionMode.PARTITIONED));
+                    op.setPhysicalOperator(new StreamLimitPOperator());
                     break;
                 }
                 case NESTEDTUPLESOURCE: {
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
index 83925cc..76b6fcf 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
@@ -34,8 +34,7 @@
     private static final long serialVersionUID = 1L;
     private IAggregateEvaluatorFactory[] aggFactories;
 
-    public SimpleAlgebricksAccumulatingAggregatorFactory(IAggregateEvaluatorFactory[] aggFactories, int[] keys,
-            int[] fdColumns) {
+    public SimpleAlgebricksAccumulatingAggregatorFactory(IAggregateEvaluatorFactory[] aggFactories, int[] keys) {
         this.aggFactories = aggFactories;
     }
 
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 3de0966..e3cd7d8 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
@@ -493,7 +493,7 @@
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) });
         IAggregateEvaluatorFactory[] aggFuns = new IAggregateEvaluatorFactory[] { new TupleCountAggregateFunctionFactory() };
         IAggregatorDescriptorFactory aggFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(aggFuns,
-                new int[] { 3 }, new int[] {});
+                new int[] { 3 });
         HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(spec, new int[] { 3 }, tpcf,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
                 aggFactory, gbyDesc, 1024);
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
index 8ac4e86..13574ea 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
@@ -36,7 +36,7 @@
             if (isIgnored(qFile.getName(), ignores))
                 continue;
 
-            if (qFile.isFile() && qFile.getName().startsWith("h11_")) {
+            if (qFile.isFile()) {
                 String resultFileName = hiveExtToResExt(qFile.getName());
                 File rFile = new File(PATH_TO_RESULTS + resultFileName);
                 testSuite.addTest(new OptimizerTestCase(qFile, rFile));
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
index eef4071..e24aaa7 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
@@ -55,7 +55,7 @@
 
 	<property>
 		<name>hive.algebricks.groupby.external</name>
-		<value>false</value>
+		<value>true</value>
 	</property>
 
 	<property>
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt
index e70ea78..d4e892a 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/ignore.txt
@@ -1 +1 @@
-q16
\ No newline at end of file
+q16_
\ No newline at end of file
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties
index 1cc34e1..f42e321 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/logging.properties
@@ -60,6 +60,5 @@
 # For example, set the com.xyz.foo logger to only log SEVERE
 # messages:
 
-edu.uci.ics.asterix.level = WARNING
-edu.uci.ics.algebricks.level = WARNING
+#edu.uci.ics.hyracks.algebricks.level = FINEST
 edu.uci.ics.hyracks.level = WARNING
diff --git a/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java b/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
index 7e4e271..a049f15 100644
--- a/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
+++ b/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
@@ -40,9 +40,7 @@
     public final static LinkedList<IAlgebraicRewriteRule> NORMALIZATION = new LinkedList<IAlgebraicRewriteRule>();

     static {

         NORMALIZATION.add(new EliminateSubplanRule());

-        NORMALIZATION.add(new IntroduceAggregateCombinerRule());

         NORMALIZATION.add(new BreakSelectIntoConjunctsRule());

-        NORMALIZATION.add(new IntroduceAggregateCombinerRule());

         NORMALIZATION.add(new PushSelectIntoJoinRule());

         NORMALIZATION.add(new ExtractGbyExpressionsRule());

         NORMALIZATION.add(new RemoveRedundantSelectRule());

@@ -84,6 +82,7 @@
         CONSOLIDATION.add(new IntroduceEarlyProjectRule());

         CONSOLIDATION.add(new ConsolidateAssignsRule());

         CONSOLIDATION.add(new IntroduceGroupByCombinerRule());

+        CONSOLIDATION.add(new IntroduceAggregateCombinerRule());

         CONSOLIDATION.add(new RemoveUnusedAssignAndAggregateRule());

     }

 

diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java
deleted file mode 100644
index 760a614..0000000
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/factory/hashfunction/MurmurHash3BinaryHashFunctionFamily.java
+++ /dev/null
@@ -1,63 +0,0 @@
-package edu.uci.ics.hivesterix.runtime.factory.hashfunction;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
-
-public class MurmurHash3BinaryHashFunctionFamily implements IBinaryHashFunctionFamily {
-
-    public static final IBinaryHashFunctionFamily INSTANCE = new MurmurHash3BinaryHashFunctionFamily();
-
-    private static final long serialVersionUID = 1L;
-
-    private MurmurHash3BinaryHashFunctionFamily() {
-    }
-
-    private static final int C1 = 0xcc9e2d51;
-    private static final int C2 = 0x1b873593;
-    private static final int C3 = 5;
-    private static final int C4 = 0xe6546b64;
-    private static final int C5 = 0x85ebca6b;
-    private static final int C6 = 0xc2b2ae35;
-
-    @Override
-    public IBinaryHashFunction createBinaryHashFunction(final int seed) {
-        return new IBinaryHashFunction() {
-            @Override
-            public int hash(byte[] bytes, int offset, int length) {
-                int h = seed;
-                int p = offset;
-                int remain = length;
-                while (remain >= 4) {
-                    int k = (bytes[p] & 0xff) | ((bytes[p + 1] & 0xff) << 8) | ((bytes[p + 2] & 0xff) << 16)
-                            | ((bytes[p + 3] & 0xff) << 24);
-                    k *= C1;
-                    k = Integer.rotateLeft(k, 15);
-                    k *= C2;
-                    h ^= k;
-                    h = Integer.rotateLeft(h, 13);
-                    h = h * C3 + C4;
-                    p += 4;
-                    remain -= 4;
-                }
-                if (remain > 0) {
-                    int k = 0;
-                    for (int i = 0; remain > 0; i += 8) {
-                        k ^= (bytes[p++] & 0xff) << i;
-                        remain--;
-                    }
-                    k *= C1;
-                    k = Integer.rotateLeft(k, 15);
-                    k *= C2;
-                    h ^= k;
-                }
-                h ^= length;
-                h ^= (h >>> 16);
-                h *= C5;
-                h ^= (h >>> 13);
-                h *= C6;
-                h ^= (h >>> 16);
-                return h;
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
index e7a2e79..45302e8 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/provider/HiveBinaryHashFunctionFamilyProvider.java
@@ -1,9 +1,9 @@
 package edu.uci.ics.hivesterix.runtime.provider;
 
-import edu.uci.ics.hivesterix.runtime.factory.hashfunction.MurmurHash3BinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+import edu.uci.ics.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
 
 public class HiveBinaryHashFunctionFamilyProvider implements IBinaryHashFunctionFamilyProvider {
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
index c4b7802..733c7d3 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
@@ -27,11 +27,9 @@
 public interface ICCApplicationContext extends IApplicationContext {
     /**
      * Sets the state that must be distributed by the infrastructure to all the
-     * NC application contexts. Any state set by calling this method in
-     * the {@link ICCApplicationEntryPoint#start(ICCApplicationContext, String[])} call
-     * is made available to all the {@link INCApplicationContext} objects at each Node Controller.
-     * The state is then available to be inspected by the application at the NC during or
-     * after the {@link INCBootstrap#start()} call.
+     * NC application contexts. Any state set by calling this method in the {@link ICCApplicationEntryPoint#start(ICCApplicationContext, String[])} call is made available to all the {@link INCApplicationContext} objects
+     * at each Node Controller. The state is then available to be inspected by
+     * the application at the NC during or after the {@link INCBootstrap#start()} call.
      * 
      * @param state
      *            The distributed state
@@ -47,6 +45,14 @@
     public void addJobLifecycleListener(IJobLifecycleListener jobLifecycleListener);
 
     /**
+     * A listener that listens to Cluster Lifecycle events at the Cluster
+     * Controller.
+     * 
+     * @param jobLifecycleListener
+     */
+    public void addClusterLifecycleListener(IClusterLifecycleListener clusterLifecycleListener);
+
+    /**
      * Get the Cluster Controller Context.
      * 
      * @return The Cluster Controller Context.
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IClusterLifecycleListener.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IClusterLifecycleListener.java
new file mode 100644
index 0000000..51db13e
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/IClusterLifecycleListener.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.application;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A listener interface for providing notification call backs to events such as a Node Controller joining/leaving the cluster.
+ */
+public interface IClusterLifecycleListener {
+
+    /**
+     * @param nodeId
+     *            A unique identifier of a Node Controller
+     * @param ncConfig
+     *            A map containing the set of configuration parameters that were used to start the Node Controller
+     */
+    public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration);
+
+    /**
+     * @param deadNodeIds
+     *            A set of Node Controller Ids that have left the cluster. The set is not cumulative.
+     */
+    public void notifyNodeFailure(Set<String> deadNodeIds);
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluator.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluator.java
new file mode 100644
index 0000000..575472c
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluator.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+
+/*
+ * Compares two tuples to make sure that records, whose comparison keys are NULL do not pass comparator filter  
+ */
+public interface IPredicateEvaluator {
+	public boolean evaluate(IFrameTupleAccessor fta0, int tupId0, IFrameTupleAccessor fta1, int tupId1);
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluatorFactory.java
similarity index 61%
copy from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluatorFactory.java
index 8f5ed64..bc2f339 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluatorFactory.java
@@ -1,21 +1,26 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * Copyright 2009-2013 by The Regents of the University of California
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- *
+ * 
  *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataset;
 
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+package edu.uci.ics.hyracks.api.dataflow.value;
 
-public interface IDatasetPartitionReader {
-    public void writeTo(IFrameWriter writer);
+import java.io.Serializable;
+
+/*
+ * Provides PredicateEvaluator for equi-join related operators 
+ */
+
+public interface IPredicateEvaluatorFactory extends Serializable {
+	public IPredicateEvaluator createPredicateEvaluator();
 }
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluatorFactoryProvider.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluatorFactoryProvider.java
new file mode 100644
index 0000000..029ab211
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IPredicateEvaluatorFactoryProvider.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import java.io.Serializable;
+
+/*
+ * Provides PredicateEvaluatorFactory based on (equi-join) keys 		
+ */
+
+public interface IPredicateEvaluatorFactoryProvider extends Serializable{
+	public IPredicateEvaluatorFactory getPredicateEvaluatorFactory(int[] keys0, int[] keys1);
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
index ae38c7f..dea077d 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
@@ -29,7 +29,10 @@
 
     public void reportPartitionFailure(JobId jobId, ResultSetId resultSetId, int partition) throws HyracksException;
 
-    public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter noc) throws HyracksException;
+    public void initializeDatasetPartitionReader(JobId jobId, ResultSetId resultSetId, int partition, IFrameWriter noc)
+            throws HyracksException;
+
+    public void abortReader(JobId jobId);
 
     public IWorkspaceFileFactory getFileFactory();
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
deleted file mode 100644
index 42dc157..0000000
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.api.dataset;
-
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public interface IDatasetPartitionWriter extends IFrameWriter {
-    public Page returnPage() throws HyracksDataException;
-}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
index d49d5cd..f29356b 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.api.dataset;
 
-import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.job.JobId;
 
 public interface IHyracksDatasetDirectoryServiceConnection {
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
index ba21a84..3fe4ada 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
@@ -14,7 +14,7 @@
  */
 package edu.uci.ics.hyracks.api.dataset;
 
-import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.job.JobId;
 
 public interface IHyracksDatasetDirectoryServiceInterface {
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
index b928a49..c397a94 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
@@ -16,7 +16,7 @@
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IHyracksDatasetReader {
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
index 095fd7d..c882448 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
@@ -17,7 +17,7 @@
 import java.net.InetSocketAddress;
 
 import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
-import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
index 47cdf97..9c3b918 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
@@ -16,7 +16,7 @@
 
 import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
 import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
-import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.job.JobId;
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
index 6419983..33e1b01 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
@@ -26,7 +26,7 @@
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
-import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.dataset.IDatasetInputChannelMonitor;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
@@ -101,7 +101,7 @@
                             knownRecords);
                     lastReadPartition = 0;
                     resultChannel = new DatasetNetworkInputChannel(netManager,
-                            getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+                            getSocketAddress(knownRecords[lastReadPartition]), jobId, resultSetId, lastReadPartition,
                             NUM_READ_BUFFERS);
                     lastMonitor = getMonitor(lastReadPartition);
                     resultChannel.open(datasetClientCtx);
@@ -118,7 +118,7 @@
 
         while (readSize <= 0 && !(isLastPartitionReadComplete())) {
             synchronized (lastMonitor) {
-                while (lastMonitor.getNFramesAvailable() <= 0 && !lastMonitor.eosReached()) {
+                while (lastMonitor.getNFramesAvailable() <= 0 && !lastMonitor.eosReached() && !lastMonitor.failed()) {
                     try {
                         lastMonitor.wait();
                     } catch (InterruptedException e) {
@@ -127,6 +127,9 @@
                 }
             }
 
+            if (lastMonitor.failed()) {
+                throw new HyracksDataException("Job Failed.");
+            }
             if (isPartitionReadComplete(lastMonitor)) {
                 knownRecords[lastReadPartition].readEOS();
                 if ((lastReadPartition == knownRecords.length - 1)) {
@@ -135,23 +138,17 @@
                     try {
                         lastReadPartition++;
                         while (knownRecords[lastReadPartition] == null) {
-                            try {
-                                knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId,
-                                        resultSetId, knownRecords);
-                            } catch (Exception e) {
-                                // Do nothing here.
-                            }
+                            knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId,
+                                    resultSetId, knownRecords);
                         }
 
                         resultChannel = new DatasetNetworkInputChannel(netManager,
-                                getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
-                                NUM_READ_BUFFERS);
+                                getSocketAddress(knownRecords[lastReadPartition]), jobId, resultSetId,
+                                lastReadPartition, NUM_READ_BUFFERS);
                         lastMonitor = getMonitor(lastReadPartition);
                         resultChannel.open(datasetClientCtx);
                         resultChannel.registerMonitor(lastMonitor);
-                    } catch (HyracksException e) {
-                        throw new HyracksDataException(e);
-                    } catch (UnknownHostException e) {
+                    } catch (Exception e) {
                         throw new HyracksDataException(e);
                     }
                 }
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
index fac2949..1ab315b 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
 import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
@@ -41,6 +42,8 @@
 
     private final JobId jobId;
 
+    private final ResultSetId resultSetId;
+
     private final int partition;
 
     private final Queue<ByteBuffer> fullQueue;
@@ -54,10 +57,11 @@
     private Object attachment;
 
     public DatasetNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, JobId jobId,
-            int partition, int nBuffers) {
+            ResultSetId resultSetId, int partition, int nBuffers) {
         this.netManager = netManager;
         this.remoteAddress = remoteAddress;
         this.jobId = jobId;
+        this.resultSetId = resultSetId;
         this.partition = partition;
         fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
         this.nBuffers = nBuffers;
@@ -103,6 +107,7 @@
         }
         ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
         writeBuffer.putLong(jobId.getId());
+        writeBuffer.putLong(resultSetId.getId());
         writeBuffer.putInt(partition);
         writeBuffer.flip();
         if (LOGGER.isLoggable(Level.FINE)) {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 6548ec7..1e78f09 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -19,6 +19,7 @@
 import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Timer;
@@ -117,6 +118,8 @@
 
     private final Map<JobId, JobRun> runMapArchive;
 
+    private final Map<JobId, List<Exception>> runMapHistory;
+
     private final WorkQueue workQueue;
 
     private ExecutorService executor;
@@ -155,6 +158,15 @@
                 return size() > ccConfig.jobHistorySize;
             }
         };
+        runMapHistory = new LinkedHashMap<JobId, List<Exception>>() {
+            private static final long serialVersionUID = 1L;
+            /** history size + 1 is for the case when history size = 0 */
+            private int allowedSize = 100 * (ccConfig.jobHistorySize + 1);
+
+            protected boolean removeEldestEntry(Map.Entry<JobId, List<Exception>> eldest) {
+                return size() > allowedSize;
+            }
+        };
         workQueue = new WorkQueue();
         this.timer = new Timer(true);
         final ClusterTopology topology = computeClusterTopology(ccConfig);
@@ -252,6 +264,10 @@
         return runMapArchive;
     }
 
+    public Map<JobId, List<Exception>> getRunHistory() {
+        return runMapHistory;
+    }
+
     public Map<String, Set<String>> getIpAddressNodeNameMap() {
         return ipAddressNodeNameMap;
     }
@@ -459,7 +475,7 @@
                 case NOTIFY_TASK_FAILURE: {
                     CCNCFunctions.NotifyTaskFailureFunction ntff = (CCNCFunctions.NotifyTaskFailureFunction) fn;
                     workQueue.schedule(new TaskFailureWork(ClusterControllerService.this, ntff.getJobId(), ntff
-                            .getTaskId(), ntff.getDetails(), ntff.getDetails()));
+                            .getTaskId(), ntff.getNodeId(), ntff.getExceptions()));
                     return;
                 }
 
@@ -554,4 +570,4 @@
     public synchronized void removeDeploymentRun(DeploymentId deploymentKey) {
         deploymentRunMap.remove(deploymentKey);
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index 7e1581a..f884c6b 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -19,9 +19,11 @@
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.application.IClusterLifecycleListener;
 import edu.uci.ics.hyracks.api.context.ICCContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
@@ -41,6 +43,7 @@
     protected IResultCallback<Object> deinitializationCallback;
 
     private List<IJobLifecycleListener> jobLifecycleListeners;
+    private List<IClusterLifecycleListener> clusterLifecycleListeners;
 
     public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext) throws IOException {
         super(serverCtx);
@@ -48,6 +51,7 @@
         initPendingNodeIds = new HashSet<String>();
         deinitPendingNodeIds = new HashSet<String>();
         jobLifecycleListeners = new ArrayList<IJobLifecycleListener>();
+        clusterLifecycleListeners = new ArrayList<IClusterLifecycleListener>();
     }
 
     public ICCContext getCCContext() {
@@ -82,4 +86,21 @@
             l.notifyJobCreation(jobId, acggf);
         }
     }
+
+    @Override
+    public void addClusterLifecycleListener(IClusterLifecycleListener clusterLifecycleListener) {
+        clusterLifecycleListeners.add(clusterLifecycleListener);
+    }
+
+    public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration) throws HyracksException {
+        for (IClusterLifecycleListener l : clusterLifecycleListeners) {
+            l.notifyNodeJoin(nodeId, ncConfiguration);
+        }
+    }
+
+    public void notifyNodeFailure(Set<String> deadNodeIds) {
+        for (IClusterLifecycleListener l : clusterLifecycleListeners) {
+            l.notifyNodeFailure(deadNodeIds);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
index e648733..21b05d4 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
@@ -99,7 +99,7 @@
         records[partition].writeEOS();
 
         for (DatasetDirectoryRecord record : records) {
-            if (record.getStatus() == DatasetDirectoryRecord.Status.SUCCESS) {
+            if ((record != null) && (record.getStatus() == DatasetDirectoryRecord.Status.SUCCESS)) {
                 successCount++;
             }
         }
@@ -112,14 +112,18 @@
     @Override
     public synchronized void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) {
         DatasetJobRecord djr = jobResultLocations.get(jobId);
-        djr.fail();
+        if (djr != null) {
+            djr.fail();
+        }
         notifyAll();
     }
 
     @Override
     public synchronized void reportJobFailure(JobId jobId) {
         DatasetJobRecord djr = jobResultLocations.get(jobId);
-        djr.fail();
+        if (djr != null) {
+            djr.fail();
+        }
         notifyAll();
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 827a71e..3a6dc26 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -78,11 +78,11 @@
 
     private JobStatus status;
 
-    private Exception exception;
+    private List<Exception> exceptions;
 
     private JobStatus pendingStatus;
 
-    private Exception pendingException;
+    private List<Exception> pendingExceptions;
 
     public JobRun(ClusterControllerService ccs, DeploymentId deploymentId, JobId jobId,
             IActivityClusterGraphGenerator acgg, EnumSet<JobFlag> jobFlags) {
@@ -124,9 +124,9 @@
         return pmm;
     }
 
-    public synchronized void setStatus(JobStatus status, Exception exception) {
+    public synchronized void setStatus(JobStatus status, List<Exception> exceptions) {
         this.status = status;
-        this.exception = exception;
+        this.exceptions = exceptions;
         notifyAll();
     }
 
@@ -134,21 +134,21 @@
         return status;
     }
 
-    public synchronized Exception getException() {
-        return exception;
+    public synchronized List<Exception> getExceptions() {
+        return exceptions;
     }
 
-    public void setPendingStatus(JobStatus status, Exception exception) {
+    public void setPendingStatus(JobStatus status, List<Exception> exceptions) {
         this.pendingStatus = status;
-        this.pendingException = exception;
+        this.pendingExceptions = exceptions;
     }
 
     public JobStatus getPendingStatus() {
         return pendingStatus;
     }
 
-    public synchronized Exception getPendingException() {
-        return pendingException;
+    public synchronized List<Exception> getPendingExceptions() {
+        return pendingExceptions;
     }
 
     public long getCreateTime() {
@@ -180,8 +180,18 @@
         while (status != JobStatus.TERMINATED && status != JobStatus.FAILURE) {
             wait();
         }
-        if (exception != null) {
-            throw new HyracksException("Job Failed", exception);
+        if (exceptions != null && !exceptions.isEmpty()) {
+            StringBuilder buffer = new StringBuilder();
+            buffer.append("Job failed on account of:\n");
+            for (Exception e : exceptions) {
+                buffer.append(e.getMessage()).append('\n');
+            }
+            HyracksException he;
+            he = new HyracksException(buffer.toString(), exceptions.get(0));
+            for (int i = 1; i < exceptions.size(); ++i) {
+                he.addSuppressed(exceptions.get(i));
+            }
+            throw he;
         }
     }
 
@@ -333,9 +343,9 @@
                                 taskAttempt.put("node-id", ta.getNodeId());
                                 taskAttempt.put("start-time", ta.getStartTime());
                                 taskAttempt.put("end-time", ta.getEndTime());
-                                String failureDetails = ta.getFailureDetails();
-                                if (failureDetails != null) {
-                                    taskAttempt.put("failure-details", failureDetails);
+                                List<Exception> exceptions = ta.getExceptions();
+                                if (exceptions != null && !exceptions.isEmpty()) {
+                                    taskAttempt.put("failure-details", exceptions);
                                 }
                                 taskAttempts.put(taskAttempt);
                             }
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
index 7c0dd57..b323501 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.cc.job;
 
+import java.util.List;
+
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 
 public class TaskAttempt {
@@ -35,7 +37,7 @@
 
     private TaskStatus status;
 
-    private String failureDetails;
+    private List<Exception> exceptions;
 
     private long startTime;
 
@@ -73,13 +75,13 @@
         return status;
     }
 
-    public String getFailureDetails() {
-        return failureDetails;
+    public List<Exception> getExceptions() {
+        return exceptions;
     }
 
-    public void setStatus(TaskStatus status, String details) {
+    public void setStatus(TaskStatus status, List<Exception> exceptions) {
         this.status = status;
-        this.failureDetails = details;
+        this.exceptions = exceptions;
     }
 
     public long getStartTime() {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index f3d7d34..d09b641 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -16,6 +16,7 @@
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -454,13 +455,13 @@
         }
     }
 
-    private void abortJob(Exception exception) {
+    private void abortJob(List<Exception> exceptions) {
         Set<TaskCluster> inProgressTaskClustersCopy = new HashSet<TaskCluster>(inProgressTaskClusters);
         for (TaskCluster tc : inProgressTaskClustersCopy) {
             abortTaskCluster(findLastTaskClusterAttempt(tc));
         }
         assert inProgressTaskClusters.isEmpty();
-        ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exception));
+        ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, jobRun.getJobId(), JobStatus.FAILURE, exceptions));
     }
 
     private void abortTaskCluster(TaskClusterAttempt tcAttempt) {
@@ -598,7 +599,7 @@
      * @param details
      *            - Cause of the failure
      */
-    public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, String details) {
+    public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, List<Exception> exceptions) {
         try {
             LOGGER.fine("Received failure notification for TaskAttempt " + ta.getTaskAttemptId());
             TaskAttemptId taId = ta.getTaskAttemptId();
@@ -606,13 +607,13 @@
             TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
             if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
                 LOGGER.fine("Marking TaskAttempt " + ta.getTaskAttemptId() + " as failed");
-                ta.setStatus(TaskAttempt.TaskStatus.FAILED, details);
+                ta.setStatus(TaskAttempt.TaskStatus.FAILED, exceptions);
                 abortTaskCluster(lastAttempt);
                 lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
                 lastAttempt.setEndTime(System.currentTimeMillis());
                 abortDoomedTaskClusters();
                 if (lastAttempt.getAttempt() >= jobRun.getActivityClusterGraph().getMaxReattempts()) {
-                    abortJob(new HyracksException(details));
+                    abortJob(exceptions);
                     return;
                 }
                 startRunnableActivityClusters();
@@ -621,7 +622,7 @@
                         + lastAttempt);
             }
         } catch (Exception e) {
-            abortJob(e);
+            abortJob(Collections.singletonList(e));
         }
     }
 
@@ -646,7 +647,10 @@
                             for (TaskAttempt ta : lastTaskClusterAttempt.getTaskAttempts().values()) {
                                 assert (ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING);
                                 if (deadNodes.contains(ta.getNodeId())) {
-                                    ta.setStatus(TaskAttempt.TaskStatus.FAILED, "Node " + ta.getNodeId() + " failed");
+                                    ta.setStatus(
+                                            TaskAttempt.TaskStatus.FAILED,
+                                            Collections.singletonList(new Exception("Node " + ta.getNodeId()
+                                                    + " failed")));
                                     ta.setEndTime(System.currentTimeMillis());
                                     abort = true;
                                 }
@@ -661,7 +665,7 @@
             }
             startRunnableActivityClusters();
         } catch (Exception e) {
-            abortJob(e);
+            abortJob(Collections.singletonList(e));
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index b304b21..ab218cc 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.List;
 import java.util.Set;
 import java.util.logging.Logger;
 
@@ -36,13 +37,13 @@
     private ClusterControllerService ccs;
     private JobId jobId;
     private JobStatus status;
-    private Exception exception;
+    private List<Exception> exceptions;
 
-    public JobCleanupWork(ClusterControllerService ccs, JobId jobId, JobStatus status, Exception exception) {
+    public JobCleanupWork(ClusterControllerService ccs, JobId jobId, JobStatus status, List<Exception> exceptions) {
         this.ccs = ccs;
         this.jobId = jobId;
         this.status = status;
-        this.exception = exception;
+        this.exceptions = exceptions;
     }
 
     @Override
@@ -58,7 +59,7 @@
         }
         Set<String> targetNodes = run.getParticipatingNodeIds();
         run.getCleanupPendingNodeIds().addAll(targetNodes);
-        run.setPendingStatus(status, exception);
+        run.setPendingStatus(status, exceptions);
         if (targetNodes != null && !targetNodes.isEmpty()) {
             for (String n : targetNodes) {
                 NodeControllerState ncs = ccs.getNodeMap().get(n);
@@ -77,9 +78,10 @@
                     e.printStackTrace();
                 }
             }
-            run.setStatus(run.getPendingStatus(), run.getPendingException());
+            run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
             ccs.getActiveRunMap().remove(jobId);
             ccs.getRunMapArchive().put(jobId, run);
+            ccs.getRunHistory().put(jobId, run.getExceptions());
             try {
                 ccs.getJobLogFile().log(createJobLogObject(run));
             } catch (Exception e) {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index 7ecdd16..0e739ba 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.Collections;
 import java.util.EnumSet;
 
 import edu.uci.ics.hyracks.api.deployment.DeploymentId;
@@ -62,7 +63,8 @@
             try {
                 run.getScheduler().startJob();
             } catch (Exception e) {
-                ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
+                ccs.getWorkQueue().schedule(
+                        new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, Collections.singletonList(e)));
             }
             callback.setValue(jobId);
         } catch (Exception e) {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index ed58c43..63e62a0 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -67,9 +67,10 @@
                     e.printStackTrace();
                 }
             }
-            run.setStatus(run.getPendingStatus(), run.getPendingException());
+            run.setStatus(run.getPendingStatus(), run.getPendingExceptions());
             ccs.getActiveRunMap().remove(jobId);
             ccs.getRunMapArchive().put(jobId, run);
+            ccs.getRunHistory().put(jobId, run.getExceptions());
             try {
                 ccs.getJobLogFile().log(createJobLogObject(run));
             } catch (Exception e) {
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
index f7dd1d2..03d43ed 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterNodeWork.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
@@ -47,6 +48,7 @@
 
         IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
         CCNCFunctions.NodeRegistrationResult result = null;
+        Map<String, String> ncConfiguration = null;
         try {
             INodeController nodeController = new NodeControllerRemoteProxy(ncIPCHandle);
 
@@ -58,6 +60,8 @@
             nodeMap.put(id, state);
             Map<String, Set<String>> ipAddressNodeNameMap = ccs.getIpAddressNodeNameMap();
             String ipAddress = state.getNCConfig().dataIPAddress;
+            ncConfiguration = new HashMap<String, String>();
+            state.getNCConfig().toMap(ncConfiguration);
             Set<String> nodes = ipAddressNodeNameMap.get(ipAddress);
             if (nodes == null) {
                 nodes = new HashSet<String>();
@@ -75,5 +79,6 @@
             result = new CCNCFunctions.NodeRegistrationResult(null, e);
         }
         ncIPCHandle.send(-1, result, null);
+        ccs.getApplicationContext().notifyNodeJoin(id, ncConfiguration);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
index 7255503..c82e264 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RemoveDeadNodesWork.java
@@ -65,6 +65,9 @@
                 }
             }
         }
+        if (deadNodes != null && deadNodes.size() > 0) {
+            ccs.getApplicationContext().notifyNodeFailure(deadNodes);
+        }
     }
 
     @Override
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
index bc8c314..51eb671 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/TaskFailureWork.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.List;
+
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.job.ActivityCluster;
 import edu.uci.ics.hyracks.api.job.JobId;
@@ -22,11 +24,12 @@
 import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
 
 public class TaskFailureWork extends AbstractTaskLifecycleWork {
-    private final String details;
+    private final List<Exception> exceptions;
 
-    public TaskFailureWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId, String details) {
+    public TaskFailureWork(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
+            List<Exception> exceptions) {
         super(ccs, jobId, taId, nodeId);
-        this.details = details;
+        this.exceptions = exceptions;
     }
 
     @Override
@@ -34,7 +37,7 @@
         JobRun run = ccs.getActiveRunMap().get(jobId);
         ccs.getDatasetDirectoryService().reportJobFailure(jobId);
         ActivityCluster ac = ta.getTask().getTaskCluster().getActivityCluster();
-        run.getScheduler().notifyTaskFailure(ta, ac, details);
+        run.getScheduler().notifyTaskFailure(ta, ac, exceptions);
     }
 
     @Override
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java
index 6cfe025..8efea17 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/WaitForJobCompletionWork.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.hyracks.control.cc.work;
 
+import java.util.List;
+
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
@@ -48,17 +50,31 @@
             });
         } else {
             final IJobStatusConditionVariable cArchivedVar = ccs.getRunMapArchive().get(jobId);
-            ccs.getExecutor().execute(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        cArchivedVar.waitForCompletion();
-                        callback.setValue(null);
-                    } catch (Exception e) {
-                        callback.setException(e);
+            if (cArchivedVar != null) {
+                ccs.getExecutor().execute(new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            cArchivedVar.waitForCompletion();
+                            callback.setValue(null);
+                        } catch (Exception e) {
+                            callback.setException(e);
+                        }
                     }
-                }
-            });
+                });
+            } else {
+                final List<Exception> exceptions = ccs.getRunHistory().get(jobId);
+                ccs.getExecutor().execute(new Runnable() {
+                    @Override
+                    public void run() {
+                        callback.setValue(null);
+                        if (exceptions != null && exceptions.size() > 0) {
+                            /** only report the first exception because IResultCallback will only throw one exception anyway */
+                            callback.setException(exceptions.get(0));
+                        }
+                    }
+                });
+            }
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 47a5c09..9954dc3 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -37,7 +37,8 @@
     public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
             throws Exception;
 
-    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception;
+    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, List<Exception> exceptions)
+            throws Exception;
 
     public void notifyJobletCleanup(JobId jobId, String nodeId) throws Exception;
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index ec29592..633d4fb 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -16,86 +16,107 @@
 
 import java.io.Serializable;
 import java.util.List;
+import java.util.Map;
 
 import org.kohsuke.args4j.Argument;
 import org.kohsuke.args4j.Option;
 import org.kohsuke.args4j.spi.StopOptionHandler;
 
 public class NCConfig implements Serializable {
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    @Option(name = "-cc-host", usage = "Cluster Controller host name", required = true)
-    public String ccHost;
+	@Option(name = "-cc-host", usage = "Cluster Controller host name", required = true)
+	public String ccHost;
 
-    @Option(name = "-cc-port", usage = "Cluster Controller port (default: 1099)")
-    public int ccPort = 1099;
+	@Option(name = "-cc-port", usage = "Cluster Controller port (default: 1099)")
+	public int ccPort = 1099;
 
-    @Option(name = "-cluster-net-ip-address", usage = "IP Address to bind cluster listener", required = true)
-    public String clusterNetIPAddress;
+	@Option(name = "-cluster-net-ip-address", usage = "IP Address to bind cluster listener", required = true)
+	public String clusterNetIPAddress;
 
-    @Option(name = "-node-id", usage = "Logical name of node controller unique within the cluster", required = true)
-    public String nodeId;
+	@Option(name = "-node-id", usage = "Logical name of node controller unique within the cluster", required = true)
+	public String nodeId;
 
-    @Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
-    public String dataIPAddress;
+	@Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
+	public String dataIPAddress;
 
-    @Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
-    public String datasetIPAddress;
+	@Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
+	public String datasetIPAddress;
 
-    @Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
-    public String ioDevices = System.getProperty("java.io.tmpdir");
+	@Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
+	public String ioDevices = System.getProperty("java.io.tmpdir");
 
-    @Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
-    public int nNetThreads = 1;
+	@Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
+	public int nNetThreads = 1;
 
-    @Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
-    public int maxMemory = -1;
+	@Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
+	public int maxMemory = -1;
 
-    @Option(name = "-result-history-size", usage = "Limits the number of jobs whose results should be remembered by the system to the specified value. (default: 10)")
-    public int resultHistorySize = 100;
+	@Option(name = "-result-history-size", usage = "Limits the number of jobs whose results should be remembered by the system to the specified value. (default: 10)")
+	public int resultHistorySize = 100;
 
-    @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
-    public int resultManagerMemory = -1;
+	@Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
+	public int resultManagerMemory = -1;
 
-    @Option(name = "-app-nc-main-class", usage = "Application NC Main Class")
-    public String appNCMainClass;
+	@Option(name = "-app-nc-main-class", usage = "Application NC Main Class")
+	public String appNCMainClass;
 
-    @Argument
-    @Option(name = "--", handler = StopOptionHandler.class)
-    public List<String> appArgs;
+	@Argument
+	@Option(name = "--", handler = StopOptionHandler.class)
+	public List<String> appArgs;
 
-    public void toCommandLine(List<String> cList) {
-        cList.add("-cc-host");
-        cList.add(ccHost);
-        cList.add("-cc-port");
-        cList.add(String.valueOf(ccPort));
-        cList.add("-cluster-net-ip-address");
-        cList.add(clusterNetIPAddress);
-        cList.add("-node-id");
-        cList.add(nodeId);
-        cList.add("-data-ip-address");
-        cList.add(dataIPAddress);
-        cList.add(datasetIPAddress);
-        cList.add("-iodevices");
-        cList.add(ioDevices);
-        cList.add("-net-thread-count");
-        cList.add(String.valueOf(nNetThreads));
-        cList.add("-max-memory");
-        cList.add(String.valueOf(maxMemory));
-        cList.add("-result-history-size");
-        cList.add(String.valueOf(resultHistorySize));
-        cList.add("-result-manager-memory");
-        cList.add(String.valueOf(resultManagerMemory));
+	public void toCommandLine(List<String> cList) {
+		cList.add("-cc-host");
+		cList.add(ccHost);
+		cList.add("-cc-port");
+		cList.add(String.valueOf(ccPort));
+		cList.add("-cluster-net-ip-address");
+		cList.add(clusterNetIPAddress);
+		cList.add("-node-id");
+		cList.add(nodeId);
+		cList.add("-data-ip-address");
+		cList.add(dataIPAddress);
+		cList.add(datasetIPAddress);
+		cList.add("-iodevices");
+		cList.add(ioDevices);
+		cList.add("-net-thread-count");
+		cList.add(String.valueOf(nNetThreads));
+		cList.add("-max-memory");
+		cList.add(String.valueOf(maxMemory));
+		cList.add("-result-history-size");
+		cList.add(String.valueOf(resultHistorySize));
+		cList.add("-result-manager-memory");
+		cList.add(String.valueOf(resultManagerMemory));
 
-        if (appNCMainClass != null) {
-            cList.add("-app-nc-main-class");
-            cList.add(appNCMainClass);
-        }
-        if (appArgs != null && !appArgs.isEmpty()) {
-            cList.add("--");
-            for (String appArg : appArgs) {
-                cList.add(appArg);
-            }
-        }
-    }
-}
+		if (appNCMainClass != null) {
+			cList.add("-app-nc-main-class");
+			cList.add(appNCMainClass);
+		}
+		if (appArgs != null && !appArgs.isEmpty()) {
+			cList.add("--");
+			for (String appArg : appArgs) {
+				cList.add(appArg);
+			}
+		}
+	}
+
+	public void toMap(Map<String, String> configuration) {
+		configuration.put("cc-host", ccHost);
+		configuration.put("cc-port", (String.valueOf(ccPort)));
+		configuration.put("cluster-net-ip-address", clusterNetIPAddress);
+		configuration.put("node-id", nodeId);
+		configuration.put("data-ip-address", dataIPAddress);
+		configuration.put("iodevices", ioDevices);
+		configuration.put("net-thread-count", String.valueOf(nNetThreads));
+		configuration.put("max-memory", String.valueOf(maxMemory));
+		configuration.put("result-history-size",
+				String.valueOf(resultHistorySize));
+		configuration.put("result-manager-memory",
+				String.valueOf(resultManagerMemory));
+
+		if (appNCMainClass != null) {
+			configuration.put("app-nc-main-class", appNCMainClass);
+		}
+
+	}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index e343657..a6382de 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -213,13 +213,13 @@
         private final JobId jobId;
         private final TaskAttemptId taskId;
         private final String nodeId;
-        private final String details;
+        private final List<Exception> exceptions;
 
-        public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, String details) {
+        public NotifyTaskFailureFunction(JobId jobId, TaskAttemptId taskId, String nodeId, List<Exception> exceptions) {
             this.jobId = jobId;
             this.taskId = taskId;
             this.nodeId = nodeId;
-            this.details = details;
+            this.exceptions = exceptions;
         }
 
         @Override
@@ -239,8 +239,8 @@
             return nodeId;
         }
 
-        public String getDetails() {
-            return details;
+        public List<Exception> getExceptions() {
+            return exceptions;
         }
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index 5ed65cc..240322c 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -59,9 +59,9 @@
     }
 
     @Override
-    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, String details) throws Exception {
+    public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, List<Exception> exceptions) throws Exception {
         CCNCFunctions.NotifyTaskFailureFunction fn = new CCNCFunctions.NotifyTaskFailureFunction(jobId, taskId, nodeId,
-                details);
+                exceptions);
         ipcHandle.send(-1, fn, null);
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index ac76c16..206bf32 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -14,13 +14,12 @@
  */
 package edu.uci.ics.hyracks.control.nc;
 
-import java.io.ByteArrayOutputStream;
-import java.io.PrintWriter;
-import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Hashtable;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Executor;
@@ -81,11 +80,7 @@
 
     private IOperatorNodePushable operator;
 
-    private volatile boolean failed;
-
-    private ByteArrayOutputStream errorBaos;
-
-    private PrintWriter errorWriter;
+    private final List<Exception> exceptions;
 
     private volatile boolean aborted;
 
@@ -102,9 +97,7 @@
         opEnv = joblet.getEnvironment();
         partitionSendProfile = new Hashtable<PartitionId, PartitionProfile>();
         pendingThreads = new LinkedHashSet<Thread>();
-        failed = false;
-        errorBaos = new ByteArrayOutputStream();
-        errorWriter = new PrintWriter(errorBaos, true);
+        exceptions = new ArrayList<>();
         this.ncs = ncs;
     }
 
@@ -252,10 +245,7 @@
                                     pushFrames(collector, writer);
                                 } catch (HyracksDataException e) {
                                     synchronized (Task.this) {
-                                        failed = true;
-                                        errorWriter.println("Exception caught by thread: " + thread.getName());
-                                        e.printStackTrace(errorWriter);
-                                        errorWriter.println();
+                                        exceptions.add(e);
                                     }
                                 } finally {
                                     thread.setName(oldName);
@@ -277,23 +267,15 @@
             NodeControllerService ncs = joblet.getNodeController();
             ncs.getWorkQueue().schedule(new NotifyTaskCompleteWork(ncs, this));
         } catch (Exception e) {
-            failed = true;
-            errorWriter.println("Exception caught by thread: " + ct.getName());
-            e.printStackTrace(errorWriter);
-            errorWriter.println();
+            exceptions.add(e);
         } finally {
             ct.setName(threadName);
             close();
             removePendingThread(ct);
         }
-        if (failed) {
-            errorWriter.close();
+        if (!exceptions.isEmpty()) {
             NodeControllerService ncs = joblet.getNodeController();
-            try {
-                ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, this, errorBaos.toString("UTF-8")));
-            } catch (UnsupportedEncodingException e) {
-                e.printStackTrace();
-            }
+            ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, this, exceptions));
         }
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
index cecd677..4e27f12 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
@@ -21,12 +21,13 @@
 import java.util.Map;
 import java.util.Set;
 
-import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
 import edu.uci.ics.hyracks.api.dataset.Page;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
 
 public class DatasetMemoryManager {
+    private int availableMemory;
+
     private final Set<Page> availPages;
 
     private final LeastRecentlyUsedList leastRecentlyUsedList;
@@ -36,29 +37,32 @@
     private final static int FRAME_SIZE = 32768;
 
     public DatasetMemoryManager(int availableMemory) {
+        this.availableMemory = availableMemory;
+
         availPages = new HashSet<Page>();
 
         // Atleast have one page for temporarily storing the results.
-        if (availableMemory <= 0)
-            availableMemory = FRAME_SIZE;
-
-        while (availableMemory >= FRAME_SIZE) {
-            /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's allocateFrame()
-             * instead of direct ByteBuffer.allocate()?
-             */
-            availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
-            availableMemory -= FRAME_SIZE;
-        }
+        if (this.availableMemory <= FRAME_SIZE)
+            this.availableMemory = FRAME_SIZE;
 
         leastRecentlyUsedList = new LeastRecentlyUsedList();
         resultPartitionNodesMap = new HashMap<ResultSetPartitionId, PartitionNode>();
     }
 
-    public Page requestPage(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter dpw)
-            throws OutOfMemoryError, HyracksDataException {
+    public synchronized Page requestPage(ResultSetPartitionId resultSetPartitionId, ResultState resultState)
+            throws HyracksDataException {
         Page page;
         if (availPages.isEmpty()) {
-            page = evictPage();
+            if (availableMemory >= FRAME_SIZE) {
+                /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's allocateFrame()
+                 * instead of direct ByteBuffer.allocate()?
+                 */
+                availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
+                availableMemory -= FRAME_SIZE;
+                page = getAvailablePage();
+            } else {
+                page = evictPage();
+            }
         } else {
             page = getAvailablePage();
         }
@@ -71,7 +75,7 @@
          * update reference call before a page is pushed on to the element of the LRU list. So we first obtain the page,
          * then make a updateReference call which in turn creates a new node in the LRU list and then add the page to it.
          */
-        PartitionNode pn = updateReference(resultSetPartitionId, dpw);
+        PartitionNode pn = updateReference(resultSetPartitionId, resultState);
         pn.add(page);
         return page;
     }
@@ -81,7 +85,7 @@
         updateReference(resultSetPartitionId, null);
     }
 
-    public int getPageSize() {
+    public static int getPageSize() {
         return FRAME_SIZE;
     }
 
@@ -90,28 +94,29 @@
         resultPartitionNodesMap.put(resultSetPartitionId, pn);
     }
 
-    protected synchronized PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId,
-            IDatasetPartitionWriter dpw) {
+    protected PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId, ResultState resultState) {
         PartitionNode pn = null;
 
         if (!resultPartitionNodesMap.containsKey(resultSetPartitionId)) {
-            if (dpw != null) {
-                pn = new PartitionNode(resultSetPartitionId, dpw);
+            if (resultState != null) {
+                pn = new PartitionNode(resultSetPartitionId, resultState);
                 insertPartitionNode(resultSetPartitionId, pn);
             }
             return pn;
         }
-        pn = resultPartitionNodesMap.get(resultSetPartitionId);
-        leastRecentlyUsedList.remove(pn);
-        insertPartitionNode(resultSetPartitionId, pn);
+        synchronized (this) {
+            pn = resultPartitionNodesMap.get(resultSetPartitionId);
+            leastRecentlyUsedList.remove(pn);
+            insertPartitionNode(resultSetPartitionId, pn);
+        }
 
         return pn;
     }
 
-    protected synchronized Page evictPage() throws HyracksDataException {
+    protected Page evictPage() throws HyracksDataException {
         PartitionNode pn = leastRecentlyUsedList.getFirst();
-        IDatasetPartitionWriter dpw = pn.getDatasetPartitionWriter();
-        Page page = dpw.returnPage();
+        ResultState resultState = pn.getResultState();
+        Page page = resultState.returnPage();
 
         /* If the partition holding the pages breaks the contract by not returning the page or it has no page, just take
          * away all the pages allocated to it and add to the available pages set.
@@ -140,7 +145,7 @@
         return page;
     }
 
-    protected synchronized Page getAvailablePage() {
+    protected Page getAvailablePage() {
         Iterator<Page> iter = availPages.iterator();
         Page page = iter.next();
         iter.remove();
@@ -197,15 +202,15 @@
 
         private final ResultSetPartitionId resultSetPartitionId;
 
-        private final IDatasetPartitionWriter datasetPartitionWriter;
+        private final ResultState resultState;
 
         private PartitionNode prev;
 
         private PartitionNode next;
 
-        public PartitionNode(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter datasetPartitionWriter) {
+        public PartitionNode(ResultSetPartitionId resultSetPartitionId, ResultState resultState) {
             this.resultSetPartitionId = resultSetPartitionId;
-            this.datasetPartitionWriter = datasetPartitionWriter;
+            this.resultState = resultState;
             prev = null;
             next = null;
         }
@@ -214,8 +219,8 @@
             return resultSetPartitionId;
         }
 
-        public IDatasetPartitionWriter getDatasetPartitionWriter() {
-            return datasetPartitionWriter;
+        public ResultState getResultState() {
+            return resultState;
         }
 
         public void setPrev(PartitionNode node) {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
index af9a607..2ec7acc 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
@@ -14,14 +14,15 @@
  */
 package edu.uci.ics.hyracks.control.nc.dataset;
 
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.concurrent.Executor;
+import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
-import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
@@ -32,11 +33,13 @@
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
 public class DatasetPartitionManager implements IDatasetPartitionManager {
+    private static final Logger LOGGER = Logger.getLogger(DatasetPartitionManager.class.getName());
+
     private final NodeControllerService ncs;
 
     private final Executor executor;
 
-    private final Map<JobId, ResultState[]> partitionResultStateMap;
+    private final Map<JobId, Map<ResultSetId, ResultState[]>> partitionResultStateMap;
 
     private final DefaultDeallocatableRegistry deallocatableRegistry;
 
@@ -50,18 +53,34 @@
         this.executor = executor;
         deallocatableRegistry = new DefaultDeallocatableRegistry();
         fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
-        datasetMemoryManager = new DatasetMemoryManager(availableMemory);
-        partitionResultStateMap = new LinkedHashMap<JobId, ResultState[]>() {
+        if (availableMemory >= DatasetMemoryManager.getPageSize()) {
+            datasetMemoryManager = new DatasetMemoryManager(availableMemory);
+        } else {
+            datasetMemoryManager = null;
+        }
+        partitionResultStateMap = new LinkedHashMap<JobId, Map<ResultSetId, ResultState[]>>() {
             private static final long serialVersionUID = 1L;
 
-            protected boolean removeEldestEntry(Map.Entry<JobId, ResultState[]> eldest) {
-                if (size() > resultHistorySize) {
-                    for (ResultState state : eldest.getValue()) {
-                        state.deinit();
+            protected boolean removeEldestEntry(Map.Entry<JobId, Map<ResultSetId, ResultState[]>> eldest) {
+                synchronized (DatasetPartitionManager.this) {
+                    if (size() > resultHistorySize) {
+                        Map<ResultSetId, ResultState[]> rsIdMap = partitionResultStateMap.get(eldest.getValue());
+                        for (ResultSetId rsId : rsIdMap.keySet()) {
+                            ResultState[] resultStates = rsIdMap.get(rsId);
+                            if (resultStates != null) {
+                                for (int i = 0; i < resultStates.length; i++) {
+                                    ResultState state = resultStates[i];
+                                    if (state != null) {
+                                        state.closeAndDelete();
+                                        LOGGER.fine("Removing partition: " + i + " for JobId: " + eldest.getKey());
+                                    }
+                                }
+                            }
+                        }
+                        return true;
                     }
-                    return true;
+                    return false;
                 }
-                return false;
             }
         };
     }
@@ -72,15 +91,21 @@
         DatasetPartitionWriter dpw = null;
         JobId jobId = ctx.getJobletContext().getJobId();
         try {
-            synchronized (partitionResultStateMap) {
+            synchronized (this) {
                 ncs.getClusterController().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
                         nPartitions, ncs.getDatasetNetworkManager().getNetworkAddress());
-                dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, partition, datasetMemoryManager);
+                dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, partition, datasetMemoryManager, fileFactory);
 
-                ResultState[] resultStates = partitionResultStateMap.get(jobId);
+                Map<ResultSetId, ResultState[]> rsIdMap = partitionResultStateMap.get(jobId);
+                if (rsIdMap == null) {
+                    rsIdMap = new HashMap<ResultSetId, ResultState[]>();
+                    partitionResultStateMap.put(jobId, rsIdMap);
+                }
+
+                ResultState[] resultStates = rsIdMap.get(rsId);
                 if (resultStates == null) {
                     resultStates = new ResultState[nPartitions];
-                    partitionResultStateMap.put(jobId, resultStates);
+                    rsIdMap.put(rsId, resultStates);
                 }
                 resultStates[partition] = dpw.getResultState();
             }
@@ -88,12 +113,15 @@
             throw new HyracksException(e);
         }
 
+        LOGGER.fine("Initialized partition writer: JobId: " + jobId + ":partition: " + partition);
         return dpw;
     }
 
     @Override
     public void reportPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
         try {
+            LOGGER.fine("Reporting partition write completion: JobId: " + jobId + ": ResultSetId: " + rsId
+                    + ":partition: " + partition);
             ncs.getClusterController().reportResultPartitionWriteCompletion(jobId, rsId, partition);
         } catch (Exception e) {
             throw new HyracksException(e);
@@ -103,6 +131,8 @@
     @Override
     public void reportPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
         try {
+            LOGGER.info("Reporting partition failure: JobId: " + jobId + ": ResultSetId: " + rsId + ":partition: "
+                    + partition);
             ncs.getClusterController().reportResultPartitionFailure(jobId, rsId, partition);
         } catch (Exception e) {
             throw new HyracksException(e);
@@ -110,24 +140,51 @@
     }
 
     @Override
-    public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter writer)
-            throws HyracksException {
+    public void initializeDatasetPartitionReader(JobId jobId, ResultSetId resultSetId, int partition,
+            IFrameWriter writer) throws HyracksException {
         ResultState resultState;
-        synchronized (partitionResultStateMap) {
-            ResultState[] resultStates = partitionResultStateMap.get(jobId);
+        synchronized (this) {
+            Map<ResultSetId, ResultState[]> rsIdMap = partitionResultStateMap.get(jobId);
 
-            if (resultStates == null) {
+            if (rsIdMap == null) {
                 throw new HyracksException("Unknown JobId " + jobId);
             }
 
+            ResultState[] resultStates = rsIdMap.get(resultSetId);
+            if (resultStates == null) {
+                throw new HyracksException("Unknown JobId: " + jobId + " ResultSetId: " + resultSetId);
+            }
+
             resultState = resultStates[partition];
             if (resultState == null) {
                 throw new HyracksException("No DatasetPartitionWriter for partition " + partition);
             }
         }
 
-        IDatasetPartitionReader dpr = new DatasetPartitionReader(datasetMemoryManager, executor, resultState);
+        DatasetPartitionReader dpr = new DatasetPartitionReader(datasetMemoryManager, executor, resultState);
         dpr.writeTo(writer);
+        LOGGER.fine("Initialized partition reader: JobId: " + jobId + ":ResultSetId: " + resultSetId + ":partition: "
+                + partition);
+    }
+
+    @Override
+    public synchronized void abortReader(JobId jobId) {
+        Map<ResultSetId, ResultState[]> rsIdMap = partitionResultStateMap.get(jobId);
+
+        if (rsIdMap == null) {
+            return;
+        }
+
+        for (ResultSetId rsId : rsIdMap.keySet()) {
+            ResultState[] resultStates = rsIdMap.get(rsId);
+            if (resultStates != null) {
+                for (ResultState state : resultStates) {
+                    if (state != null) {
+                        state.abort();
+                    }
+                }
+            }
+        }
     }
 
     @Override
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
index a584b4b..07624de 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
@@ -20,14 +20,10 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
-import edu.uci.ics.hyracks.api.dataset.Page;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.IFileHandle;
-import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
 
-public class DatasetPartitionReader implements IDatasetPartitionReader {
+public class DatasetPartitionReader {
     private static final Logger LOGGER = Logger.getLogger(DatasetPartitionReader.class.getName());
 
     private final DatasetMemoryManager datasetMemoryManager;
@@ -36,51 +32,12 @@
 
     private final ResultState resultState;
 
-    private IFileHandle fileHandle;
-
     public DatasetPartitionReader(DatasetMemoryManager datasetMemoryManager, Executor executor, ResultState resultState) {
         this.datasetMemoryManager = datasetMemoryManager;
         this.executor = executor;
         this.resultState = resultState;
     }
 
-    private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
-        long readSize = 0;
-        synchronized (resultState) {
-            while (offset >= resultState.getSize() && !resultState.getEOS()) {
-                try {
-                    resultState.wait();
-                } catch (InterruptedException e) {
-                    throw new HyracksDataException(e);
-                }
-            }
-        }
-
-        if (offset >= resultState.getSize() && resultState.getEOS()) {
-            return readSize;
-        }
-
-        if (offset < resultState.getPersistentSize()) {
-            readSize = resultState.getIOManager().syncRead(fileHandle, offset, buffer);
-        }
-
-        if (readSize < buffer.capacity()) {
-            long localPageOffset = offset - resultState.getPersistentSize();
-            int localPageIndex = (int) (localPageOffset / datasetMemoryManager.getPageSize());
-            int pageOffset = (int) (localPageOffset % datasetMemoryManager.getPageSize());
-            Page page = resultState.getPage(localPageIndex);
-            if (page == null) {
-            	return readSize;
-            }
-            readSize += buffer.remaining();
-            buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
-        }
-
-        datasetMemoryManager.pageReferenced(resultState.getResultSetPartitionId());
-        return readSize;
-    }
-
-    @Override
     public void writeTo(final IFrameWriter writer) {
         executor.execute(new Runnable() {
             @Override
@@ -88,8 +45,7 @@
                 NetworkOutputChannel channel = (NetworkOutputChannel) writer;
                 channel.setFrameSize(resultState.getFrameSize());
                 try {
-                    fileHandle = resultState.getIOManager().open(resultState.getValidFileReference(),
-                            IIOManager.FileReadWriteMode.READ_ONLY, IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                    resultState.readOpen();
                     channel.open();
                     try {
                         long offset = 0;
@@ -109,10 +65,8 @@
                         }
                     } finally {
                         channel.close();
-                        resultState.getIOManager().close(fileHandle);
+                        resultState.readClose();
                     }
-                } catch (InterruptedException e) {
-                    throw new RuntimeException(e);
                 } catch (HyracksDataException e) {
                     throw new RuntimeException(e);
                 }
@@ -120,6 +74,14 @@
                     LOGGER.info("result reading successful(" + resultState.getResultSetPartitionId() + ")");
                 }
             }
+
+            private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+                if (datasetMemoryManager == null) {
+                    return resultState.read(offset, buffer);
+                } else {
+                    return resultState.read(datasetMemoryManager, offset, buffer);
+                }
+            }
         });
     }
 }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
index 317f553..8f4b639 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
@@ -18,24 +18,19 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
-import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
-import edu.uci.ics.hyracks.api.dataset.Page;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IFileHandle;
-import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
 
-public class DatasetPartitionWriter implements IDatasetPartitionWriter {
+public class DatasetPartitionWriter implements IFrameWriter {
     private static final Logger LOGGER = Logger.getLogger(DatasetPartitionWriter.class.getName());
 
-    private static final String FILE_PREFIX = "result_";
-
     private final IDatasetPartitionManager manager;
 
     private final JobId jobId;
@@ -50,10 +45,9 @@
 
     private final ResultState resultState;
 
-    private IFileHandle fileHandle;
-
     public DatasetPartitionWriter(IHyracksTaskContext ctx, IDatasetPartitionManager manager, JobId jobId,
-            ResultSetId rsId, int partition, DatasetMemoryManager datasetMemoryManager) {
+            ResultSetId rsId, int partition, DatasetMemoryManager datasetMemoryManager,
+            IWorkspaceFileFactory fileFactory) {
         this.manager = manager;
         this.jobId = jobId;
         this.resultSetId = rsId;
@@ -61,7 +55,7 @@
         this.datasetMemoryManager = datasetMemoryManager;
 
         resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
-        resultState = new ResultState(resultSetPartitionId, ctx.getIOManager(), ctx.getFrameSize());
+        resultState = new ResultState(resultSetPartitionId, ctx.getIOManager(), fileFactory, ctx.getFrameSize());
     }
 
     public ResultState getResultState() {
@@ -69,41 +63,27 @@
     }
 
     @Override
-    public void open() throws HyracksDataException {
+    public void open() {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("open(" + partition + ")");
         }
-        String fName = FILE_PREFIX + String.valueOf(partition);
-        FileReference fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(fName);
-        fileHandle = resultState.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-        resultState.init(fRef, fileHandle);
+        resultState.open();
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        int srcOffset = 0;
-        Page destPage = resultState.getLastPage();
-
-        while (srcOffset < buffer.limit()) {
-            if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
-                destPage = datasetMemoryManager.requestPage(resultSetPartitionId, this);
-                resultState.addPage(destPage);
-            }
-            int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
-            destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
-            srcOffset += srcLength;
-            resultState.incrementSize(srcLength);
-        }
-
-        synchronized (resultState) {
-            resultState.notifyAll();
+        if (datasetMemoryManager == null) {
+            resultState.write(buffer);
+        } else {
+            resultState.write(datasetMemoryManager, buffer);
         }
     }
 
     @Override
     public void fail() throws HyracksDataException {
         try {
+            resultState.closeAndDelete();
+            resultState.abort();
             manager.reportPartitionFailure(jobId, resultSetId, partition);
         } catch (HyracksException e) {
             throw new HyracksDataException(e);
@@ -117,32 +97,10 @@
         }
 
         try {
-            synchronized (resultState) {
-                resultState.setEOS(true);
-                resultState.notifyAll();
-            }
+            resultState.close();
             manager.reportPartitionWriteCompletion(jobId, resultSetId, partition);
         } catch (HyracksException e) {
             throw new HyracksDataException(e);
         }
     }
-
-    @Override
-    public Page returnPage() throws HyracksDataException {
-        Page page = resultState.removePage(0);
-
-        IIOManager ioManager = resultState.getIOManager();
-
-        // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
-        if (page == null) {
-            ioManager.close(fileHandle);
-            return null;
-        }
-
-        page.getBuffer().flip();
-
-        long delta = ioManager.syncWrite(fileHandle, resultState.getPersistentSize(), page.getBuffer());
-        resultState.incrementPersistentSize(delta);
-        return page;
-    }
 }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
index 661df93..911f372 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
@@ -17,28 +17,35 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
 import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IFileHandle;
 import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
 
 public class ResultState implements IStateObject {
+    private static final String FILE_PREFIX = "result_";
+
     private final ResultSetPartitionId resultSetPartitionId;
 
     private final int frameSize;
 
     private final IIOManager ioManager;
 
+    private final IWorkspaceFileFactory fileFactory;
+
     private final AtomicBoolean eos;
 
-    private final AtomicBoolean readEOS;
+    private final AtomicBoolean failed;
 
     private final List<Page> localPageList;
 
@@ -46,29 +53,40 @@
 
     private IFileHandle writeFileHandle;
 
+    private IFileHandle readFileHandle;
+
     private long size;
 
     private long persistentSize;
 
-    ResultState(ResultSetPartitionId resultSetPartitionId, IIOManager ioManager, int frameSize) {
+    ResultState(ResultSetPartitionId resultSetPartitionId, IIOManager ioManager, IWorkspaceFileFactory fileFactory,
+            int frameSize) {
         this.resultSetPartitionId = resultSetPartitionId;
         this.ioManager = ioManager;
+        this.fileFactory = fileFactory;
         this.frameSize = frameSize;
         eos = new AtomicBoolean(false);
-        readEOS = new AtomicBoolean(false);
+        failed = new AtomicBoolean(false);
         localPageList = new ArrayList<Page>();
+
+        fileRef = null;
+        writeFileHandle = null;
     }
 
-    public synchronized void init(FileReference fileRef, IFileHandle writeFileHandle) {
-        this.fileRef = fileRef;
-        this.writeFileHandle = writeFileHandle;
-
+    public synchronized void open() {
         size = 0;
         persistentSize = 0;
+    }
+
+    public synchronized void close() {
+        eos.set(true);
         notifyAll();
     }
 
-    public synchronized void deinit() {
+    public synchronized void closeAndDelete() {
+        // Deleting a job is equivalent to aborting the job for all practical purposes, so the same action, needs
+        // to be taken when there are more requests to these result states.
+        failed.set(true);
         if (writeFileHandle != null) {
             try {
                 ioManager.close(writeFileHandle);
@@ -76,7 +94,149 @@
                 // Since file handle could not be closed, just ignore.
             }
         }
-        fileRef.delete();
+        if (fileRef != null) {
+            fileRef.delete();
+        }
+    }
+
+    public synchronized void write(ByteBuffer buffer) throws HyracksDataException {
+        if (fileRef == null) {
+            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
+            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
+            writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        }
+
+        size += ioManager.syncWrite(writeFileHandle, size, buffer);
+
+        notifyAll();
+    }
+
+    public synchronized void write(DatasetMemoryManager datasetMemoryManager, ByteBuffer buffer)
+            throws HyracksDataException {
+        int srcOffset = 0;
+        Page destPage = null;
+
+        if (!localPageList.isEmpty()) {
+            destPage = localPageList.get(localPageList.size() - 1);
+        }
+
+        while (srcOffset < buffer.limit()) {
+            if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
+                destPage = datasetMemoryManager.requestPage(resultSetPartitionId, this);
+                localPageList.add(destPage);
+            }
+            int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
+            destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
+            srcOffset += srcLength;
+            size += srcLength;
+        }
+
+        notifyAll();
+    }
+
+    public synchronized void readOpen() {
+        // It is a noOp for now, leaving here to keep the API stable for future usage.
+    }
+
+    public synchronized void readClose() throws HyracksDataException {
+        if (readFileHandle != null) {
+            ioManager.close(readFileHandle);
+        }
+    }
+
+    public synchronized long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+        long readSize = 0;
+
+        while (offset >= size && !eos.get() && !failed.get()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        if ((offset >= size && eos.get()) || failed.get()) {
+            return readSize;
+        }
+
+        if (readFileHandle == null) {
+            initReadFileHandle();
+        }
+        readSize = ioManager.syncRead(readFileHandle, offset, buffer);
+
+        return readSize;
+    }
+
+    public long read(DatasetMemoryManager datasetMemoryManager, long offset, ByteBuffer buffer)
+            throws HyracksDataException {
+        long readSize = 0;
+        synchronized (this) {
+            while (offset >= size && !eos.get() && !failed.get()) {
+                try {
+                    wait();
+                } catch (InterruptedException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            if ((offset >= size && eos.get()) || failed.get()) {
+                return readSize;
+            }
+
+            if (offset < persistentSize) {
+                if (readFileHandle == null) {
+                    initReadFileHandle();
+                }
+                readSize = ioManager.syncRead(readFileHandle, offset, buffer);
+            }
+
+            if (readSize < buffer.capacity()) {
+                long localPageOffset = offset - persistentSize;
+                int localPageIndex = (int) (localPageOffset / DatasetMemoryManager.getPageSize());
+                int pageOffset = (int) (localPageOffset % DatasetMemoryManager.getPageSize());
+                Page page = getPage(localPageIndex);
+                if (page == null) {
+                    return readSize;
+                }
+                readSize += buffer.remaining();
+                buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
+            }
+        }
+        datasetMemoryManager.pageReferenced(resultSetPartitionId);
+        return readSize;
+    }
+
+    public synchronized void abort() {
+        failed.set(true);
+        notifyAll();
+    }
+
+    public synchronized Page returnPage() throws HyracksDataException {
+        Page page = removePage();
+
+        // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
+        if (page == null) {
+            ioManager.close(writeFileHandle);
+            return null;
+        }
+
+        page.getBuffer().flip();
+
+        if (fileRef == null) {
+            String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
+            fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
+            writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+            notifyAll();
+        }
+
+        long delta = ioManager.syncWrite(writeFileHandle, persistentSize, page.getBuffer());
+        persistentSize += delta;
+        return page;
+    }
+
+    public synchronized void setEOS(boolean eos) {
+        this.eos.set(eos);
     }
 
     public ResultSetPartitionId getResultSetPartitionId() {
@@ -91,76 +251,6 @@
         return ioManager;
     }
 
-    public synchronized void incrementSize(long delta) {
-        size += delta;
-    }
-
-    public synchronized long getSize() {
-        return size;
-    }
-
-    public synchronized void incrementPersistentSize(long delta) {
-        persistentSize += delta;
-    }
-
-    public synchronized long getPersistentSize() {
-        return persistentSize;
-    }
-
-    public void setEOS(boolean eos) {
-        this.eos.set(eos);
-    }
-
-    public boolean getEOS() {
-        return eos.get();
-    }
-
-    public boolean getReadEOS() {
-        return readEOS.get();
-    }
-
-    public synchronized void addPage(Page page) {
-        localPageList.add(page);
-    }
-
-    public synchronized Page removePage(int index) {
-        Page page = null;
-        if (!localPageList.isEmpty()) {
-            page = localPageList.remove(index);
-        }
-        return page;
-    }
-
-    public synchronized Page getPage(int index) {
-        Page page = null;
-        if (!localPageList.isEmpty()) {
-            page = localPageList.get(index);
-        }
-        return page;
-    }
-
-    public synchronized Page getLastPage() {
-        Page page = null;
-        if (!localPageList.isEmpty()) {
-            page = localPageList.get(localPageList.size() - 1);
-        }
-        return page;
-    }
-
-    public synchronized Page getFirstPage() {
-        Page page = null;
-        if (!localPageList.isEmpty()) {
-            page = localPageList.get(0);
-        }
-        return page;
-    }
-
-    public synchronized FileReference getValidFileReference() throws InterruptedException {
-        while (fileRef == null)
-            wait();
-        return fileRef;
-    }
-
     @Override
     public JobId getJobId() {
         return resultSetPartitionId.getJobId();
@@ -185,4 +275,36 @@
     public void fromBytes(DataInput in) throws IOException {
         throw new UnsupportedOperationException();
     }
+
+    private Page getPage(int index) {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.get(index);
+        }
+        return page;
+    }
+
+    private Page removePage() {
+        Page page = null;
+        if (!localPageList.isEmpty()) {
+            page = localPageList.remove(localPageList.size() - 1);
+        }
+        return page;
+    }
+
+    private void initReadFileHandle() throws HyracksDataException {
+        while (fileRef == null && !failed.get()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        if (failed.get()) {
+            return;
+        }
+
+        readFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_ONLY,
+                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
index 5b8b333..84baf49 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
@@ -24,6 +24,7 @@
 
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
@@ -95,6 +96,7 @@
         @Override
         public void accept(ByteBuffer buffer) {
             JobId jobId = new JobId(buffer.getLong());
+            ResultSetId rsId = new ResultSetId(buffer.getLong());
             int partition = buffer.getInt();
             if (LOGGER.isLoggable(Level.FINE)) {
                 LOGGER.fine("Received initial dataset partition read request for JobId: " + jobId + " partition: "
@@ -102,7 +104,7 @@
             }
             noc = new NetworkOutputChannel(ccb, 1);
             try {
-                partitionManager.initializeDatasetPartitionReader(jobId, partition, noc);
+                partitionManager.initializeDatasetPartitionReader(jobId, rsId, partition, noc);
             } catch (HyracksException e) {
                 noc.abort();
             }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/AbortTasksWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/AbortTasksWork.java
index 8f8c032..54ac99a 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/AbortTasksWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/AbortTasksWork.java
@@ -20,6 +20,7 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 import edu.uci.ics.hyracks.control.nc.Joblet;
@@ -46,6 +47,11 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Aborting Tasks: " + jobId + ":" + tasks);
         }
+        IDatasetPartitionManager dpm = ncs.getDatasetPartitionManager();
+        if (dpm != null) {
+            ncs.getDatasetPartitionManager().abortReader(jobId);
+        }
+
         Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
         Joblet ji = jobletMap.get(jobId);
         if (ji != null) {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java
index 3957934..c70a1bd 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/NotifyTaskFailureWork.java
@@ -14,6 +14,10 @@
  */
 package edu.uci.ics.hyracks.control.nc.work;
 
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
 import edu.uci.ics.hyracks.control.nc.Task;
@@ -21,19 +25,23 @@
 public class NotifyTaskFailureWork extends AbstractWork {
     private final NodeControllerService ncs;
     private final Task task;
-    private final String details;
+    private final List<Exception> exceptions;
 
-    public NotifyTaskFailureWork(NodeControllerService ncs, Task task, String details) {
+    public NotifyTaskFailureWork(NodeControllerService ncs, Task task, List<Exception> exceptions) {
         this.ncs = ncs;
         this.task = task;
-        this.details = details;
+        this.exceptions = exceptions;
     }
 
     @Override
     public void run() {
         try {
-            ncs.getClusterController().notifyTaskFailure(task.getJobletContext().getJobId(), task.getTaskAttemptId(),
-                    ncs.getId(), details);
+            JobId jobId = task.getJobletContext().getJobId();
+            IDatasetPartitionManager dpm = ncs.getDatasetPartitionManager();
+            if (dpm != null) {
+                dpm.abortReader(jobId);
+            }
+            ncs.getClusterController().notifyTaskFailure(jobId, task.getTaskAttemptId(), ncs.getId(), exceptions);
         } catch (Exception e) {
             e.printStackTrace();
         }
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/MurmurHash3BinaryHashFunctionFamily.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/MurmurHash3BinaryHashFunctionFamily.java
index 8f2e32c..588ad00 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/MurmurHash3BinaryHashFunctionFamily.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/accessors/MurmurHash3BinaryHashFunctionFamily.java
@@ -3,9 +3,23 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 
-public class MurmurHash3BinaryHashFunctionFamily implements IBinaryHashFunctionFamily {
+/**
+ * An implementation of the Murmur3 hash family. The code is implemented based
+ * on the original <a
+ * href=http://code.google.com/p/guava-libraries/source/browse
+ * /guava/src/com/google/common/hash/Murmur3_32HashFunction.java>guava
+ * implementation</a> from Google Guava library.
+ */
+public class MurmurHash3BinaryHashFunctionFamily implements
+        IBinaryHashFunctionFamily {
+
+    public static final IBinaryHashFunctionFamily INSTANCE = new MurmurHash3BinaryHashFunctionFamily();
+
     private static final long serialVersionUID = 1L;
 
+    private MurmurHash3BinaryHashFunctionFamily() {
+    }
+
     private static final int C1 = 0xcc9e2d51;
     private static final int C2 = 0x1b873593;
     private static final int C3 = 5;
@@ -21,9 +35,10 @@
                 int h = seed;
                 int p = offset;
                 int remain = length;
-                while (remain > 4) {
-                    int k = ((int) bytes[p]) | (((int) bytes[p + 1]) << 8) | (((int) bytes[p + 2]) << 16)
-                            | (((int) bytes[p + 3]) << 24);
+                while (remain >= 4) {
+                    int k = (bytes[p] & 0xff) | ((bytes[p + 1] & 0xff) << 8)
+                            | ((bytes[p + 2] & 0xff) << 16)
+                            | ((bytes[p + 3] & 0xff) << 24);
                     k *= C1;
                     k = Integer.rotateLeft(k, 15);
                     k *= C2;
@@ -33,20 +48,16 @@
                     p += 4;
                     remain -= 4;
                 }
-                int k = 0;
-                switch (remain) {
-                    case 3:
-                        k = bytes[p++];
-                    case 2:
-                        k = (k << 8) | bytes[p++];
-                    case 1:
-                        k = (k << 8) | bytes[p++];
-                        k *= C1;
-                        k = Integer.rotateLeft(k, 15);
-                        k *= C2;
-                        h ^= k;
-                        h = Integer.rotateLeft(h, 13);
-                        h = h * C3 + C4;
+                if (remain > 0) {
+                    int k = 0;
+                    for (int i = 0; remain > 0; i += 8) {
+                        k ^= (bytes[p++] & 0xff) << i;
+                        remain--;
+                    }
+                    k *= C1;
+                    k = Integer.rotateLeft(k, 15);
+                    k *= C2;
+                    h ^= k;
                 }
                 h ^= length;
                 h ^= (h >>> 16);
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
index 07f6ba2..c403501 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
@@ -32,6 +32,9 @@
     }
 
     public void reset(ByteBuffer buffer, boolean clear) {
+        if (clear) {
+            buffer.clear();
+        }
         frameTupleAppender.reset(buffer, clear);
     }
 
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index cbe2ae6..bebef21 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -88,7 +88,7 @@
             flushFrame();
             tupleAppender.reset(buffer, true);
             if (!tupleAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException();
+                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + buffer.capacity() + ")");
             }
         }
     }
@@ -103,4 +103,4 @@
     public void fail() throws HyracksDataException {
         frameWriter.fail();
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index 2841509..5e5cae1 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -207,7 +207,7 @@
                                 FrameUtils.flushFrame(outBuffer, writer);
                                 appender.reset(outBuffer, true);
                                 if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                    throw new IllegalStateException();
+                                    throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + outBuffer.capacity() + ")");
                                 }
                             }
                         }
@@ -235,4 +235,4 @@
             }
         };
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
index 4b1144a..e498975 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
@@ -114,7 +114,7 @@
                     runGen.nextFrame(frame);
                     fta.reset(frame, true);
                     if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                        throw new IllegalStateException();
+                        throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + frame.capacity() + ")");
                     }
                 }
             }
@@ -268,4 +268,4 @@
             }
         };
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
index 33d58af..1a6f6fd 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
@@ -114,7 +114,7 @@
         FrameUtils.copy(buffer, copyFrame);
     }
 
-    private void accumulate(FrameTupleAccessor accessor, int tIndex) {
+    private void accumulate(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
         if (!fta.append(accessor, tIndex)) {
             ++bPtr;
             if (group.size() <= bPtr) {
@@ -122,7 +122,9 @@
             }
             fta.reset(group.get(bPtr), true);
             if (!fta.append(accessor, tIndex)) {
-                throw new IllegalStateException();
+                throw new HyracksDataException("Record size ("
+                        + (accessor.getTupleEndOffset(tIndex) - accessor.getTupleStartOffset(tIndex))
+                        + ") larger than frame size (" + group.get(bPtr).capacity() + ")");
             }
         }
     }
@@ -184,4 +186,4 @@
     @Override
     public void fail() throws HyracksDataException {
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
index 8e03eae..253dcbf 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
@@ -154,7 +154,9 @@
             if (!fta.append(accessor, tIdx)) {
                 flush();
                 if (!fta.append(accessor, tIdx)) {
-                    throw new IllegalStateException();
+                    throw new HyracksDataException("Record size ("
+                            + (accessor.getTupleEndOffset(tIdx) - accessor.getTupleStartOffset(tIdx))
+                            + ") larger than frame size (" + fta.getBuffer().capacity() + ")");
                 }
             }
         }
@@ -175,4 +177,4 @@
             fta.reset(buffer, true);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-std/pom.xml b/hyracks/hyracks-dataflow-std/pom.xml
index 56046f7..8e67891 100644
--- a/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks/hyracks-dataflow-std/pom.xml
@@ -45,5 +45,10 @@
   		<type>jar</type>
   		<scope>test</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>commons-io</groupId>
+  		<artifactId>commons-io</artifactId>
+  		<version>2.4</version>
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
index 6ec9013..bea94ff 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
@@ -83,7 +83,7 @@
                 flushFrame(appenderBuffer, pWriters[h]);
                 appender.reset(appenderBuffer, true);
                 if (!appender.append(tupleAccessor, i)) {
-                    throw new IllegalStateException();
+                    throw new HyracksDataException("Record size (" + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i)) + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                 }
             }
         }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index 6b3f3c0..820e2ce 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -87,7 +87,7 @@
                 flushFrame(appenderBuffer, pWriters[h]);
                 appender.reset(appenderBuffer, true);
                 if (!appender.append(tupleAccessor, i)) {
-                    throw new IllegalStateException();
+                    throw new HyracksDataException("Record size (" + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i)) + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                 }
             }
         }
@@ -99,4 +99,4 @@
             pWriters[i].fail();
         }
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 7cf437d..9c1ea9d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -71,7 +71,7 @@
                             FrameUtils.flushFrame(frame, writer);
                             appender.reset(frame, true);
                             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                throw new IllegalStateException();
+                                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                             }
                         }
                     }
@@ -249,4 +249,4 @@
             return true;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileRemoveOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileRemoveOperatorDescriptor.java
new file mode 100644
index 0000000..bf7ff33
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileRemoveOperatorDescriptor.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.commons.io.FileUtils;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+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;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+
+public class FileRemoveOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private final IFileSplitProvider fileSplitProvider;
+
+    public FileRemoveOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvder) {
+        super(spec, 0, 0);
+        this.fileSplitProvider = fileSplitProvder;
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        final FileSplit split = fileSplitProvider.getFileSplits()[partition];
+        return new AbstractOperatorNodePushable() {
+
+            @Override
+            public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                throw new IllegalStateException();
+            }
+
+            @Override
+            public void initialize() throws HyracksDataException {
+                File f = split.getLocalFile().getFile();
+                try {
+                    FileUtils.deleteDirectory(f);
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public IFrameWriter getInputFrameWriter(int index) {
+                throw new IllegalStateException();
+            }
+
+            @Override
+            public int getInputArity() {
+                return 0;
+            }
+
+            @Override
+            public void deinitialize() throws HyracksDataException {
+            }
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
index b74d97b..35649ca 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
@@ -23,17 +23,25 @@
     private static final long serialVersionUID = 1L;
 
     private final String nodeName;
-
     private final FileReference file;
+    private final int ioDeviceId;
 
     public FileSplit(String nodeName, FileReference file) {
         this.nodeName = nodeName;
         this.file = file;
+        this.ioDeviceId = 0;
+    }
+
+    public FileSplit(String nodeName, FileReference file, int ioDeviceId) {
+        this.nodeName = nodeName;
+        this.file = file;
+        this.ioDeviceId = ioDeviceId;
     }
 
     public FileSplit(String nodeName, String path) {
         this.nodeName = nodeName;
         this.file = new FileReference(new File(path));
+        this.ioDeviceId = 0;
     }
 
     public String getNodeName() {
@@ -43,4 +51,8 @@
     public FileReference getLocalFile() {
         return file;
     }
+
+    public int getIODeviceId() {
+        return ioDeviceId;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
index 6c58d6f..0bb514e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
@@ -22,6 +22,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -42,13 +44,14 @@
     private final double factor;
     private final IBinaryHashFunctionFactory[] hashFunctionFactories;
     private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IPredicateEvaluatorFactory predEvaluatorFactory;
     private final boolean isLeftOuter;
     private final INullWriterFactory[] nullWriterFactories1;
 
     public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             int recordsPerFrame, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor) {
+            RecordDescriptor recordDescriptor, IPredicateEvaluatorFactory predEvalFactory) {
         super(spec, 2, 1);
         this.memsize = memsize;
         this.inputsize0 = inputsize0;
@@ -58,6 +61,7 @@
         this.keys1 = keys1;
         this.hashFunctionFactories = hashFunctionFactories;
         this.comparatorFactories = comparatorFactories;
+        this.predEvaluatorFactory = predEvalFactory;
         this.isLeftOuter = false;
         this.nullWriterFactories1 = null;
         recordDescriptors[0] = recordDescriptor;
@@ -66,7 +70,7 @@
     public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             int recordsPerFrame, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
+            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1, IPredicateEvaluatorFactory predEvalFactory) {
         super(spec, 2, 1);
         this.memsize = memsize;
         this.inputsize0 = inputsize0;
@@ -76,6 +80,7 @@
         this.keys1 = keys1;
         this.hashFunctionFactories = hashFunctionFactories;
         this.comparatorFactories = comparatorFactories;
+        this.predEvaluatorFactory = predEvalFactory;
         this.isLeftOuter = isLeftOuter;
         this.nullWriterFactories1 = nullWriterFactories1;
         recordDescriptors[0] = recordDescriptor;
@@ -143,12 +148,13 @@
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(rpartAid, 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(spartAid, 0);
             int numPartitions = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
-
+            final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator() );
+            
             return new GraceHashJoinOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
                     RPARTITION_ACTIVITY_ID), partition), new TaskId(new ActivityId(getOperatorId(),
                     SPARTITION_ACTIVITY_ID), partition), recordsPerFrame, factor, keys0, keys1, hashFunctionFactories,
                     comparatorFactories, nullWriterFactories1, rd1, rd0, recordDescriptors[0], numPartitions,
-                    isLeftOuter);
+                    predEvaluator, isLeftOuter);
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
index 91f509d..7c9bd88 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -51,12 +52,13 @@
     private final double factor;
     private final int numPartitions;
     private final boolean isLeftOuter;
+    private final IPredicateEvaluator predEvaluator;
 
     GraceHashJoinOperatorNodePushable(IHyracksTaskContext ctx, Object state0Id, Object state1Id, int recordsPerFrame,
             double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
             IBinaryComparatorFactory[] comparatorFactories, INullWriterFactory[] nullWriterFactories,
             RecordDescriptor rd1, RecordDescriptor rd0, RecordDescriptor outRecordDescriptor, int numPartitions,
-            boolean isLeftOuter) {
+            IPredicateEvaluator predEval, boolean isLeftOuter) {
         this.ctx = ctx;
         this.state0Id = state0Id;
         this.state1Id = state1Id;
@@ -70,6 +72,7 @@
         this.numPartitions = numPartitions;
         this.recordsPerFrame = recordsPerFrame;
         this.factor = factor;
+        this.predEvaluator = predEval;
         this.isLeftOuter = isLeftOuter;
     }
 
@@ -114,7 +117,7 @@
                 table.reset();
                 InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(
                         ctx.getFrameSize(), rd0), hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1), hpcRep1,
-                        new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1, table);
+                        new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1, table, predEvaluator);
 
                 // build
                 if (buildWriter != null) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 4f9b987..ea9414b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -29,6 +29,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
@@ -66,6 +68,7 @@
     private final int[] keys1;
     private final IBinaryHashFunctionFactory[] hashFunctionFactories;
     private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IPredicateEvaluatorFactory predEvaluatorFactory;
     private final boolean isLeftOuter;
     private final INullWriterFactory[] nullWriterFactories1;
 
@@ -87,7 +90,7 @@
     public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             int recordsPerFrame, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor) throws HyracksDataException {
+            RecordDescriptor recordDescriptor, IPredicateEvaluatorFactory predEvalFactory) throws HyracksDataException {
         super(spec, 2, 1);
         this.memsize = memsize;
         this.inputsize0 = inputsize0;
@@ -97,6 +100,7 @@
         this.keys1 = keys1;
         this.hashFunctionFactories = hashFunctionFactories;
         this.comparatorFactories = comparatorFactories;
+        this.predEvaluatorFactory = predEvalFactory;
         this.isLeftOuter = false;
         this.nullWriterFactories1 = null;
         recordDescriptors[0] = recordDescriptor;
@@ -105,7 +109,7 @@
     public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             int recordsPerFrame, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1)
+            RecordDescriptor recordDescriptor, IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1)
             throws HyracksDataException {
         super(spec, 2, 1);
         this.memsize = memsize;
@@ -116,6 +120,7 @@
         this.keys1 = keys1;
         this.hashFunctionFactories = hashFunctionFactories;
         this.comparatorFactories = comparatorFactories;
+        this.predEvaluatorFactory = predEvalFactory;
         this.isLeftOuter = isLeftOuter;
         this.nullWriterFactories1 = nullWriterFactories1;
         recordDescriptors[0] = recordDescriptor;
@@ -189,6 +194,7 @@
                     nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
                 }
             }
+            final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
 
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
@@ -315,7 +321,7 @@
                     state.joiner = new InMemoryHashJoin(ctx, tableSize,
                             new FrameTupleAccessor(ctx.getFrameSize(), rd0), hpc0, new FrameTupleAccessor(
                                     ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
-                                    comparators), isLeftOuter, nullWriters1, table);
+                                    comparators), isLeftOuter, nullWriters1, table, predEvaluator);
                     bufferForPartitions = new ByteBuffer[state.nPartitions];
                     state.fWriters = new RunFileWriter[state.nPartitions];
                     for (int i = 0; i < state.nPartitions; i++) {
@@ -377,6 +383,7 @@
                     nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
                 }
             }
+            final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
 
             IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
                 private BuildAndPartitionTaskState state;
@@ -500,7 +507,7 @@
                             InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(
                                     ctx.getFrameSize(), rd0), hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1),
                                     hpcRep1, new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter,
-                                    nullWriters1, table);
+                                    nullWriters1, table, predEvaluator);
 
                             if (buildWriter != null) {
                                 RunFileReader buildReader = buildWriter.createReader();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index e1fda74..0f27aa7 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 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.IPredicateEvaluator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -47,18 +48,19 @@
     private final int tableSize;
     private final TuplePointer storedTuplePointer;
     private final boolean reverseOutputOrder; //Should we reverse the order of tuples, we are writing in output
+    private final IPredicateEvaluator predEvaluator;
 
     public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
             ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
             FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1,
-            ISerializableTable table) throws HyracksDataException {
-        this(ctx, tableSize, accessor0, tpc0, accessor1, tpc1, comparator, isLeftOuter, nullWriters1, table, false);
+            ISerializableTable table, IPredicateEvaluator predEval) throws HyracksDataException {
+        this(ctx, tableSize, accessor0, tpc0, accessor1, tpc1, comparator, isLeftOuter, nullWriters1, table, predEval, false);
     }
 
     public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
             ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
             FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1,
-            ISerializableTable table, boolean reverse) throws HyracksDataException {
+            ISerializableTable table, IPredicateEvaluator predEval, boolean reverse) throws HyracksDataException {
         this.tableSize = tableSize;
         this.table = table;
         storedTuplePointer = new TuplePointer();
@@ -71,6 +73,7 @@
         tpComparator = comparator;
         outBuffer = ctx.allocateFrame();
         appender.reset(outBuffer, true);
+        predEvaluator = predEval;
         this.isLeftOuter = isLeftOuter;
         if (isLeftOuter) {
             int fieldCountOuter = accessor1.getFieldCount();
@@ -103,35 +106,38 @@
         accessorProbe.reset(buffer);
         int tupleCount0 = accessorProbe.getTupleCount();
         for (int i = 0; i < tupleCount0; ++i) {
-            int entry = tpcProbe.partition(accessorProbe, i, tableSize);
-            boolean matchFound = false;
-            int offset = 0;
-            do {
-                table.getTuplePointer(entry, offset++, storedTuplePointer);
-                if (storedTuplePointer.frameIndex < 0)
-                    break;
-                int bIndex = storedTuplePointer.frameIndex;
-                int tIndex = storedTuplePointer.tupleIndex;
-                accessorBuild.reset(buffers.get(bIndex));
-                int c = tpComparator.compare(accessorProbe, i, accessorBuild, tIndex);
-                if (c == 0) {
-                    matchFound = true;
-                    appendToResult(i, tIndex, writer);
-                }
-            } while (true);
-
+        	boolean matchFound = false;
+        	if(tableSize != 0){
+        		int entry = tpcProbe.partition(accessorProbe, i, tableSize);
+                int offset = 0;
+                do {
+                    table.getTuplePointer(entry, offset++, storedTuplePointer);
+                    if (storedTuplePointer.frameIndex < 0)
+                        break;
+                    int bIndex = storedTuplePointer.frameIndex;
+                    int tIndex = storedTuplePointer.tupleIndex;
+                    accessorBuild.reset(buffers.get(bIndex));
+                    int c = tpComparator.compare(accessorProbe, i, accessorBuild, tIndex);
+                    if (c == 0) {
+                    	boolean predEval = ( (predEvaluator == null) || predEvaluator.evaluate(accessorProbe, i, accessorBuild, tIndex) );
+                    	if(predEval){
+                    		matchFound = true;
+                            appendToResult(i, tIndex, writer);
+                    	}
+                    }
+                } while (true);
+        	}
             if (!matchFound && isLeftOuter) {
-
                 if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
                         nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
                     flushFrame(outBuffer, writer);
                     appender.reset(outBuffer, true);
                     if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
                             nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
-                        throw new IllegalStateException();
+                        throw new HyracksDataException("Record size larger than frame size ("
+                                + appender.getBuffer().capacity() + ")");
                     }
                 }
-
             }
         }
     }
@@ -156,7 +162,12 @@
                 flushFrame(outBuffer, writer);
                 appender.reset(outBuffer, true);
                 if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
-                    throw new IllegalStateException();
+                    int tSize = accessorProbe.getTupleEndOffset(probeSidetIx)
+                            - accessorProbe.getTupleStartOffset(probeSidetIx)
+                            + accessorBuild.getTupleEndOffset(buildSidetIx)
+                            - accessorBuild.getTupleStartOffset(buildSidetIx);
+                    throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
+                            + appender.getBuffer().capacity() + ")");
                 }
             }
         } else {
@@ -164,9 +175,14 @@
                 flushFrame(outBuffer, writer);
                 appender.reset(outBuffer, true);
                 if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
-                    throw new IllegalStateException();
+                    int tSize = accessorProbe.getTupleEndOffset(probeSidetIx)
+                            - accessorProbe.getTupleStartOffset(probeSidetIx)
+                            + accessorBuild.getTupleEndOffset(buildSidetIx)
+                            - accessorBuild.getTupleStartOffset(buildSidetIx);
+                    throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
+                            + appender.getBuffer().capacity() + ")");
                 }
             }
         }
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index e0a5613..705923a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -29,6 +29,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -53,18 +55,20 @@
     private final int[] keys1;
     private final IBinaryHashFunctionFactory[] hashFunctionFactories;
     private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IPredicateEvaluatorFactory predEvaluatorFactory;
     private final boolean isLeftOuter;
     private final INullWriterFactory[] nullWriterFactories1;
     private final int tableSize;
 
     public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor, int tableSize) {
+            RecordDescriptor recordDescriptor, int tableSize, IPredicateEvaluatorFactory predEvalFactory) {
         super(spec, 2, 1);
         this.keys0 = keys0;
         this.keys1 = keys1;
         this.hashFunctionFactories = hashFunctionFactories;
         this.comparatorFactories = comparatorFactories;
+        this.predEvaluatorFactory = predEvalFactory;
         recordDescriptors[0] = recordDescriptor;
         this.isLeftOuter = false;
         this.nullWriterFactories1 = null;
@@ -73,18 +77,34 @@
 
     public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
+            IPredicateEvaluatorFactory predEvalFactory, RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
             int tableSize) {
         super(spec, 2, 1);
         this.keys0 = keys0;
         this.keys1 = keys1;
         this.hashFunctionFactories = hashFunctionFactories;
         this.comparatorFactories = comparatorFactories;
+        this.predEvaluatorFactory = predEvalFactory;
         recordDescriptors[0] = recordDescriptor;
         this.isLeftOuter = isLeftOuter;
         this.nullWriterFactories1 = nullWriterFactories1;
         this.tableSize = tableSize;
     }
+    
+    public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, int tableSize) {
+    	this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories, recordDescriptor, tableSize, null);
+    }
+    
+    public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
+            int tableSize) {
+    	this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories,null,recordDescriptor,isLeftOuter,nullWriterFactories1,tableSize);
+    }
+    
+    
 
     @Override
     public void contributeActivities(IActivityGraphBuilder builder) {
@@ -150,6 +170,7 @@
                     nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
                 }
             }
+            final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
 
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private HashBuildTaskState state;
@@ -166,7 +187,7 @@
                     state.joiner = new InMemoryHashJoin(ctx, tableSize,
                             new FrameTupleAccessor(ctx.getFrameSize(), rd0), hpc0, new FrameTupleAccessor(
                                     ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
-                                    comparators), isLeftOuter, nullWriters1, table);
+                                    comparators), isLeftOuter, nullWriters1, table, predEvaluator);
                 }
 
                 @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index 6870e71..9496b76 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 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.IPredicateEvaluator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -47,9 +48,10 @@
     private final RunFileWriter runFileWriter;
     private final boolean isLeftOuter;
     private final ArrayTupleBuilder nullTupleBuilder;
-
+    private final IPredicateEvaluator predEvaluator;
+    
     public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
-            ITuplePairComparator comparators, int memSize, boolean isLeftOuter, INullWriter[] nullWriters1)
+            ITuplePairComparator comparators, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter, INullWriter[] nullWriters1)
             throws HyracksDataException {
         this.accessorInner = accessor1;
         this.accessorOuter = accessor0;
@@ -60,6 +62,7 @@
         this.appender.reset(outBuffer, true);
         this.outBuffers = new ArrayList<ByteBuffer>();
         this.memSize = memSize;
+        this.predEvaluator = predEval;
         this.ctx = ctx;
 
         this.isLeftOuter = isLeftOuter;
@@ -130,13 +133,17 @@
             boolean matchFound = false;
             for (int j = 0; j < tupleCount1; ++j) {
                 int c = compare(accessorOuter, i, accessorInner, j);
-                if (c == 0) {
-                    matchFound = true;
+                boolean prdEval = (predEvaluator == null) || (predEvaluator.evaluate(accessorOuter, i, accessorInner, j));
+                if (c == 0 && prdEval) {
+                	matchFound = true;
                     if (!appender.appendConcat(accessorOuter, i, accessorInner, j)) {
                         flushFrame(outBuffer, writer);
                         appender.reset(outBuffer, true);
                         if (!appender.appendConcat(accessorOuter, i, accessorInner, j)) {
-                            throw new IllegalStateException();
+                            int tSize = accessorOuter.getTupleEndOffset(i) - accessorOuter.getTupleStartOffset(i)
+                                    + accessorInner.getTupleEndOffset(j) - accessorInner.getTupleStartOffset(j);
+                            throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
+                                    + appender.getBuffer().capacity() + ")");
                         }
                     }
                 }
@@ -149,7 +156,10 @@
                     appender.reset(outBuffer, true);
                     if (!appender.appendConcat(accessorOuter, i, nullTupleBuilder.getFieldEndOffsets(),
                             nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize())) {
-                        throw new IllegalStateException();
+                        int tSize = accessorOuter.getTupleEndOffset(i) - accessorOuter.getTupleStartOffset(i)
+                                + nullTupleBuilder.getSize();
+                        throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
+                                + appender.getBuffer().capacity() + ")");
                     }
                 }
             }
@@ -196,4 +206,4 @@
         }
         return 0;
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index 0be01c1..d3f664e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -27,6 +27,8 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
@@ -49,16 +51,24 @@
     private static final long serialVersionUID = 1L;
     private final ITuplePairComparatorFactory comparatorFactory;
     private final int memSize;
+    private final IPredicateEvaluatorFactory predEvaluatorFactory;
     private final boolean isLeftOuter;
     private final INullWriterFactory[] nullWriterFactories1;
-
+    
     public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
             ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
             boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
+        this(spec, comparatorFactory, recordDescriptor, memSize, null, isLeftOuter, nullWriterFactories1);
+    }
+    
+    public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
+            ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
+            IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
         super(spec, 2, 1);
         this.comparatorFactory = comparatorFactory;
         this.recordDescriptors[0] = recordDescriptor;
         this.memSize = memSize;
+        this.predEvaluatorFactory = predEvalFactory;
         this.isLeftOuter = isLeftOuter;
         this.nullWriterFactories1 = nullWriterFactories1;
     }
@@ -117,7 +127,8 @@
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
-
+            final IPredicateEvaluator predEvaluator = ( (predEvaluatorFactory != null) ? predEvaluatorFactory.createPredicateEvaluator() : null);
+            
             final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
             if (isLeftOuter) {
                 for (int i = 0; i < nullWriterFactories1.length; i++) {
@@ -134,7 +145,7 @@
                             partition));
 
                     state.joiner = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
-                            new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize, isLeftOuter,
+                            new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize, predEvaluator, isLeftOuter,
                             nullWriters1);
 
                 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index c81bf54..993ad5a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -9,6 +9,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -53,7 +54,8 @@
 
     private RunFileWriter[] buildRFWriters; //writing spilled build partitions
     private RunFileWriter[] probeRFWriters; //writing spilled probe partitions
-
+    
+    private final IPredicateEvaluator predEvaluator;
     private final boolean isLeftOuter;
     private final INullWriter[] nullWriters1;
 
@@ -78,15 +80,15 @@
     private ByteBuffer[] sPartBuffs;    //Buffers for probe spilled partitions (one buffer per spilled partition)
     private ByteBuffer probeResBuff;    //Buffer for probe resident partition tuples
     private ByteBuffer reloadBuffer;    //Buffer for reloading spilled partitions during partition tuning 
-
+    
     private int[] buildPSizeInFrames; //Used for partition tuning
     private int freeFramesCounter; //Used for partition tuning
     
     private boolean isTableEmpty;	//Added for handling the case, where build side is empty (tableSize is 0)
-
+    
     public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
             String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
-            RecordDescriptor probeRd, ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc) {
+            RecordDescriptor probeRd, ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc, IPredicateEvaluator predEval) {
         this.ctx = ctx;
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
@@ -106,6 +108,7 @@
         this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
         this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
 
+        this.predEvaluator = predEval;
         this.isLeftOuter = false;
         this.nullWriters1 = null;
 
@@ -114,7 +117,7 @@
     public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
             String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
             RecordDescriptor probeRd, ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc,
-            boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
+            IPredicateEvaluator predEval, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
         this.ctx = ctx;
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
@@ -133,7 +136,8 @@
 
         this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
         this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
-
+        
+        this.predEvaluator = predEval;
         this.isLeftOuter = isLeftOuter;
 
         this.nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
@@ -423,7 +427,7 @@
         this.inMemJoiner = new InMemoryHashJoin(ctx, inMemTupCount,
                 new FrameTupleAccessor(ctx.getFrameSize(), probeRd), probeHpc, new FrameTupleAccessor(
                         ctx.getFrameSize(), buildRd), buildHpc, new FrameTuplePairComparator(probeKeys, buildKeys,
-                        comparators), isLeftOuter, nullWriters1, table);
+                        comparators), isLeftOuter, nullWriters1, table, predEvaluator);
     }
 
     private void cacheInMemJoin() throws HyracksDataException {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index cf39416..01c331a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -30,8 +30,9 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
+import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -112,7 +113,8 @@
     private final IBinaryComparatorFactory[] comparatorFactories; //For in-mem HJ
     private final ITuplePairComparatorFactory tuplePairComparatorFactory0; //For NLJ in probe
     private final ITuplePairComparatorFactory tuplePairComparatorFactory1; //For NLJ in probe
-
+    private final IPredicateEvaluatorFactory predEvaluatorFactory;
+    
     private final boolean isLeftOuter;
     private final INullWriterFactory[] nullWriterFactories1;
 
@@ -120,7 +122,7 @@
             double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
             ITuplePairComparatorFactory tupPaircomparatorFactory0,
-            ITuplePairComparatorFactory tupPaircomparatorFactory1, boolean isLeftOuter,
+            ITuplePairComparatorFactory tupPaircomparatorFactory1, IPredicateEvaluatorFactory predEvaluatorFactory, boolean isLeftOuter,
             INullWriterFactory[] nullWriterFactories1) throws HyracksDataException {
 
         super(spec, 2, 1);
@@ -134,6 +136,7 @@
         this.tuplePairComparatorFactory0 = tupPaircomparatorFactory0;
         this.tuplePairComparatorFactory1 = tupPaircomparatorFactory1;
         recordDescriptors[0] = recordDescriptor;
+        this.predEvaluatorFactory = predEvaluatorFactory;
         this.isLeftOuter = isLeftOuter;
         this.nullWriterFactories1 = nullWriterFactories1;
         
@@ -143,7 +146,7 @@
     public OptimizedHybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
-            ITuplePairComparatorFactory tupPaircomparatorFactory0, ITuplePairComparatorFactory tupPaircomparatorFactory1)
+            ITuplePairComparatorFactory tupPaircomparatorFactory0, ITuplePairComparatorFactory tupPaircomparatorFactory1, IPredicateEvaluatorFactory predEvaluatorFactory)
             throws HyracksDataException {
 
         super(spec, 2, 1);
@@ -156,6 +159,7 @@
         this.comparatorFactories = comparatorFactories;
         this.tuplePairComparatorFactory0 = tupPaircomparatorFactory0;
         this.tuplePairComparatorFactory1 = tupPaircomparatorFactory1;
+        this.predEvaluatorFactory = predEvaluatorFactory;
         recordDescriptors[0] = recordDescriptor;
         this.isLeftOuter = false;
         this.nullWriterFactories1 = null;
@@ -256,6 +260,8 @@
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
             
+            final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
+            
             
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
@@ -277,12 +283,12 @@
                     if(!isLeftOuter){
                     	state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
                                 PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
-                                buildHpc);
+                                buildHpc, predEvaluator);
                     }
                     else{
                     	state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
                                 PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
-                                buildHpc, isLeftOuter, nullWriterFactories1);
+                                buildHpc, predEvaluator, isLeftOuter, nullWriterFactories1);
                     }
                     
                     state.hybridHJ.initBuild();
@@ -336,7 +342,8 @@
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
             final ITuplePairComparator nljComparator0 = tuplePairComparatorFactory0.createTuplePairComparator(ctx);
             final ITuplePairComparator nljComparator1 = tuplePairComparatorFactory1.createTuplePairComparator(ctx);
-
+            final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
+            
             for (int i = 0; i < comparatorFactories.length; i++) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
@@ -372,9 +379,7 @@
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                	if(!state.hybridHJ.isTableEmpty()){
-                		state.hybridHJ.probe(buffer, writer);
-                	}
+                	state.hybridHJ.probe(buffer, writer);
                 }
 
                 @Override
@@ -459,7 +464,7 @@
                                     nPartitions);
                            
                             rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, PROBE_REL, BUILD_REL,
-                                    probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc, buildHpc);
+                                    probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc, buildHpc, predEvaluator);
 
                             buildSideReader.open();
                             rHHj.initBuild();
@@ -521,7 +526,7 @@
                                     nPartitions);
                             
                             rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, BUILD_REL, PROBE_REL,
-                                    buildKeys, probeKeys, comparators, buildRd, probeRd, buildHpc, probeHpc);
+                                    buildKeys, probeKeys, comparators, buildRd, probeRd, buildHpc, probeHpc, predEvaluator);
 
                             probeSideReader.open();
                             rHHj.initBuild();
@@ -589,7 +594,7 @@
                     InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(
                             ctx.getFrameSize(), probeRDesc), hpcRepLarger, new FrameTupleAccessor(ctx.getFrameSize(),
                             buildRDesc), hpcRepSmaller, new FrameTuplePairComparator(pKeys, bKeys, comparators),
-                            isLeftOuter, nullWriters1, table, reverse);
+                            isLeftOuter, nullWriters1, table, predEvaluator, reverse);
 
                     bReader.open();
                     rPartbuff.clear();
@@ -617,7 +622,7 @@
                         throws HyracksDataException {
 
                     NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), outerRd),
-                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize, false, null);
+                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize, predEvaluator, false, null);
 
                     ByteBuffer cacheBuff = ctx.allocateFrame();
                     innerReader.open();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index b51132e..39cfb88 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -194,7 +194,9 @@
                     outFrameAppender.reset(outFrame, true);
                     for (int i = 0; i < copyCount; i++) {
                         if (!outFrameAppender.append(fta, i)) {
-                            throw new IllegalStateException();
+                            throw new HyracksDataException("Record size ("
+                                    + (fta.getTupleEndOffset(i) - fta.getTupleStartOffset(i))
+                                    + ") larger than frame size (" + outFrameAppender.getBuffer().capacity() + ")");
                         }
                         totalCount++;
                     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
index 680b98e..ff02d57 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -132,7 +132,8 @@
                 FrameUtils.flushFrame(outFrame, writer);
                 appender.reset(outFrame, true);
                 if (!appender.append(fta1, tStart, tEnd)) {
-                    throw new IllegalStateException();
+                    throw new HyracksDataException("Record size (" + (tEnd - tStart) + ") larger than frame size ("
+                            + appender.getBuffer().capacity() + ")");
                 }
             }
         }
@@ -242,4 +243,4 @@
     public void close() {
         this.buffers.clear();
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index f342d23..b9a27ec 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -90,7 +90,7 @@
                             FrameUtils.flushFrame(outputFrame, writer);
                             appender.reset(outputFrame, true);
                             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                throw new IllegalStateException();
+                                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                             }
                         }
                     }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
index 935724b..932e166 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/common/TreeOperatorTestHelper.java
@@ -25,6 +25,7 @@
     protected final String sep = System.getProperty("file.separator");
     protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
     protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+    protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
 
     public String getPrimaryIndexName() {
         return System.getProperty("java.io.tmpdir") + sep + "primary" + simpleDateFormat.format(new Date());
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
index 62d4362..dfd5495 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/PartitionedWordInvertedIndexTest.java
@@ -54,7 +54,7 @@
         invertedIndexDataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
                 new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
                 SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE,
-                DEFAULT_MEM_NUM_PAGES);
+                DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 
     @Override
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
index c35c6c9..9e5a47b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
@@ -48,7 +48,8 @@
 
         invertedIndexDataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(new ConstantMergePolicyProvider(
                 MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
-                SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+                SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE,
+                DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 
     @Override
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index 912ab0e..c67cbfd 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -35,7 +35,8 @@
     public IIndexDataflowHelperFactory createDataFlowHelperFactory() {
         return new LSMBTreeDataflowHelperFactory(new ConstantMergePolicyProvider(MERGE_THRESHOLD),
                 ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
-                NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+                NoOpIOOperationCallback.INSTANCE, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 
 }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index 84b34b7..a892b4e 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -42,6 +42,6 @@
         return new LSMRTreeDataflowHelperFactory(valueProviderFactories, rtreePolicyType, btreeComparatorFactories,
                 new ConstantMergePolicyProvider(MERGE_THRESHOLD), ThreadCountingOperationTrackerFactory.INSTANCE,
                 SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory,
-                DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES);
+                DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE);
     }
 }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index e74e54c..89679ae 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -60,9 +60,10 @@
     private static IHyracksClientConnection hcc;
 
     private final List<File> outputFiles;
-    
+
     protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
     protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+    protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
 
     @Rule
     public TemporaryFolder outputFolder = new TemporaryFolder();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java
index 0999ee8..b7fd241 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java
@@ -90,7 +90,7 @@
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 custOrderJoinDesc, new JoinComparatorFactory(
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
-                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0));
+                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
@@ -168,7 +168,7 @@
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 custOrderJoinDesc, new JoinComparatorFactory(
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
-                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0));
+                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
@@ -247,7 +247,7 @@
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 custOrderJoinDesc, new JoinComparatorFactory(
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
-                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0));
+                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index b5eb850..9b35867 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -122,7 +122,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, 128);
+                custOrderJoinDesc, 128, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -205,7 +205,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc);
+                custOrderJoinDesc, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -288,7 +288,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc);
+                custOrderJoinDesc, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -372,7 +372,7 @@
                 new int[] { 1 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, true, nullWriterFactories, 128);
+                null, custOrderJoinDesc, true, nullWriterFactories, 128);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -460,7 +460,7 @@
                 new int[] { 1 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, true, nullWriterFactories);
+                custOrderJoinDesc, true, nullWriterFactories, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -548,7 +548,7 @@
                 new int[] { 1 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, true, nullWriterFactories);
+                custOrderJoinDesc, null, true, nullWriterFactories);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -629,7 +629,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, 128);
+                custOrderJoinDesc, 128, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -720,7 +720,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc);
+                custOrderJoinDesc, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -811,7 +811,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc);
+                custOrderJoinDesc, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -898,7 +898,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, 128);
+                custOrderJoinDesc, 128, null);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -991,7 +991,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, 128);
+                custOrderJoinDesc, 128, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         ResultSetId rsId = new ResultSetId(1);
diff --git a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
index 249e3bb..cda1380 100644
--- a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
+++ b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
@@ -44,7 +44,7 @@
                             FrameUtils.flushFrame(frame, writer);
                             appender.reset(frame, true);
                             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                throw new IllegalStateException();
+                                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                             }
                         }
                     }
@@ -167,4 +167,4 @@
             return true;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 6df9ff8..8dab4de 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -210,7 +210,7 @@
                     new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                             .of(UTF8StringPointable.FACTORY) },
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    custOrderJoinDesc);
+                    custOrderJoinDesc, null);
 
         } else if ("hybridhash".equalsIgnoreCase(algo)) {
             join = new HybridHashJoinOperatorDescriptor(
@@ -224,7 +224,7 @@
                     new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                             .of(UTF8StringPointable.FACTORY) },
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    custOrderJoinDesc);
+                    custOrderJoinDesc, null);
 
         } else {
             join = new InMemoryHashJoinOperatorDescriptor(
@@ -234,7 +234,7 @@
                     new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                             .of(UTF8StringPointable.FACTORY) },
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    custOrderJoinDesc, 6000000);
+                    custOrderJoinDesc, 6000000, null);
         }
 
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index 648e523..35ec2f6 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -17,6 +17,7 @@
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -67,7 +68,8 @@
                 FrameUtils.flushFrame(writeBuffer, writer);
                 appender.reset(writeBuffer, true);
                 if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    throw new IllegalStateException();
+                    throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
+                            + appender.getBuffer().capacity() + ")");
                 }
             }
         }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index 354aa1e..5027dc7 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -82,6 +82,16 @@
     }
 
     @Override
+    public int getMaxTupleSize(int pageSize) {
+        return (pageSize - getPageHeaderSize()) / 2;
+    }
+
+    @Override
+    public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+        return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize();
+    }
+
+    @Override
     public void setPage(ICachedPage page) {
         this.page = page;
         this.buf = page.getBuffer();
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
index 90b167f..8753d6b 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java
@@ -54,6 +54,11 @@
     }
 
     @Override
+    public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+        return tupleWriter.bytesRequired(tuple) + childPtrSize + slotManager.getSlotSize();
+    }
+
+    @Override
     public void initBuffer(byte level) {
         super.initBuffer(level);
         buf.putInt(rightLeafOff, -1);
@@ -185,7 +190,7 @@
         ITreeIndexFrame targetFrame = null;
 
         int totalSize = 0;
-        int halfPageSize = buf.capacity() / 2 - getPageHeaderSize();
+        int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
         int i;
         for (i = 0; i < tupleCount; ++i) {
             frameTuple.resetByTupleIndex(this, i);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
index 04b3077..bfb1eca 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
@@ -46,6 +46,11 @@
     }
 
     @Override
+    public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+        return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize();
+    }
+    
+    @Override
     public void initBuffer(byte level) {
         super.initBuffer(level);
         buf.putInt(nextLeafOff, -1);
@@ -146,7 +151,7 @@
         int tuplesToLeft;
         ITreeIndexFrame targetFrame = null;
         int totalSize = 0;
-        int halfPageSize = buf.capacity() / 2 - getPageHeaderSize();
+        int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
         int i;
         for (i = 0; i < tupleCount; ++i) {
             frameTuple.resetByTupleIndex(this, i);
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 86bc32a..8846873 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -47,6 +47,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ISplitKey;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
@@ -74,6 +75,7 @@
 
     private final AtomicInteger smoCounter;
     private final ReadWriteLock treeLatch;
+    private final int maxTupleSize;
 
     public BTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, IFreePageManager freePageManager,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
@@ -82,6 +84,10 @@
                 fieldCount, file);
         this.treeLatch = new ReentrantReadWriteLock(true);
         this.smoCounter = new AtomicInteger();
+        ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+        ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+        maxTupleSize = Math.min(leafFrame.getMaxTupleSize(bufferCache.getPageSize()),
+                interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
     }
 
     private void diskOrderScan(ITreeIndexCursor icursor, BTreeOpContext ctx) throws HyracksDataException {
@@ -304,11 +310,23 @@
     }
 
     private void insert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+        int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+                ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+        if (tupleSize > maxTupleSize) {
+            throw new TreeIndexException("Space required for record (" + tupleSize
+                    + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+        }
         ctx.modificationCallback.before(tuple);
         insertUpdateOrDelete(tuple, ctx);
     }
 
     private void upsert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+        int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+                ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+        if (tupleSize > maxTupleSize) {
+            throw new TreeIndexException("Space required for record (" + tupleSize
+                    + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+        }
         ctx.modificationCallback.before(tuple);
         insertUpdateOrDelete(tuple, ctx);
     }
@@ -320,6 +338,12 @@
         if (fieldCount == ctx.cmp.getKeyFieldCount()) {
             throw new BTreeNotUpdateableException("Cannot perform updates when the entire tuple forms the key.");
         }
+        int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+                ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+        if (tupleSize > maxTupleSize) {
+            throw new TreeIndexException("Space required for record (" + tupleSize
+                    + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+        }
         ctx.modificationCallback.before(tuple);
         insertUpdateOrDelete(tuple, ctx);
     }
@@ -933,6 +957,13 @@
         @Override
         public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
             try {
+                int tupleSize = Math.max(leafFrame.getBytesRequriedToWriteTuple(tuple),
+                        interiorFrame.getBytesRequriedToWriteTuple(tuple));
+                if (tupleSize > maxTupleSize) {
+                    throw new TreeIndexException("Space required for record (" + tupleSize
+                            + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+                }
+
                 NodeFrontier leafFrontier = nodeFrontiers.get(0);
 
                 int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
index 612af25..4a14505 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrame.java
@@ -59,6 +59,10 @@
 
     public ByteBuffer getBuffer();
 
+    public int getMaxTupleSize(int pageSize);
+
+    public int getBytesRequriedToWriteTuple(ITupleReference tuple);
+
     // for debugging
     public String printHeader();
 
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
index e46efff..22c6637 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
+import java.io.File;
 import java.io.IOException;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -23,6 +24,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexFileNameUtil;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
 import edu.uci.ics.hyracks.storage.common.file.LocalResource;
@@ -37,6 +39,7 @@
     protected final ResourceIdFactory resourceIdFactory;
     protected final FileReference file;
     protected final int partition;
+    protected final int ioDeviceId;
 
     protected IIndex index;
 
@@ -47,7 +50,9 @@
         this.localResourceRepository = opDesc.getStorageManager().getLocalResourceRepository(ctx);
         this.resourceIdFactory = opDesc.getStorageManager().getResourceIdFactory(ctx);
         this.partition = partition;
-        this.file = opDesc.getFileSplitProvider().getFileSplits()[partition].getLocalFile();
+        this.ioDeviceId = opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId();
+        this.file = new FileReference(new File(IndexFileNameUtil.prepareFileName(opDesc.getFileSplitProvider()
+                .getFileSplits()[partition].getLocalFile().getFile().getPath(), ioDeviceId)));
     }
 
     protected abstract IIndex createIndexInstance() throws HyracksDataException;
@@ -70,7 +75,7 @@
             // any physical artifact that the LocalResourceRepository is managing (e.g. a file containing the resource ID). 
             // Once the index has been created, a new resource ID can be generated.
             if (resourceID != -1) {
-                localResourceRepository.deleteResourceByName(file.getFile().getPath());
+                localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
             }
             index.create();
             try {
@@ -78,8 +83,9 @@
                 resourceID = resourceIdFactory.createId();
                 ILocalResourceFactory localResourceFactory = opDesc.getLocalResourceFactoryProvider()
                         .getLocalResourceFactory();
-                localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
-                        .getPath(), partition));
+                localResourceRepository.insert(
+                        localResourceFactory.createLocalResource(resourceID, file.getFile().getPath(), partition),
+                        ioDeviceId);
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
@@ -121,7 +127,7 @@
             }
 
             if (resourceID != -1) {
-                localResourceRepository.deleteResourceByName(file.getFile().getPath());
+                localResourceRepository.deleteResourceByName(file.getFile().getPath(), ioDeviceId);
             }
             index.destroy();
         }
@@ -143,4 +149,4 @@
     public IHyracksTaskContext getTaskContext() {
         return ctx;
     }
-}
+}
\ No newline at end of file
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 98beea2..d867c6c 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
@@ -119,7 +119,7 @@
                 FrameUtils.flushFrame(writeBuffer, writer);
                 appender.reset(writeBuffer, true);
                 if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    throw new IllegalStateException();
+                    throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                 }
             }
         }
@@ -162,4 +162,4 @@
     public void fail() throws HyracksDataException {
         writer.fail();
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index a861af7..c2b5ca8 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -80,7 +80,7 @@
                         FrameUtils.flushFrame(frame, writer);
                         appender.reset(frame, true);
                         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                            throw new IllegalStateException();
+                            throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                         }
                     }
                 }
@@ -104,4 +104,4 @@
     public void deinitialize() throws HyracksDataException {
         treeIndexHelper.close();
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index 09d357d..7e57240 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -79,7 +79,7 @@
             UTF8StringSerializerDeserializer.INSTANCE.serialize(stats.toString(), dos);
             tb.addFieldEndOffset();
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException();
+                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
             }
             FrameUtils.flushFrame(frame, writer);
         } catch (Exception e) {
@@ -89,4 +89,4 @@
             treeIndexHelper.close();
         }
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
index 31ce573..2e97b21 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
@@ -61,6 +61,11 @@
     }
 
     @Override
+    public int getMaxTupleSize(int pageSize) {
+        return (pageSize - getPageHeaderSize()) / 2;
+    }
+
+    @Override
     public boolean isLeaf() {
         return buf.get(levelOff) == 0;
     }
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexFileNameUtil.java
similarity index 63%
copy from hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
copy to hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexFileNameUtil.java
index 8f5ed64..fbab3cf 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexFileNameUtil.java
@@ -1,21 +1,25 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * Copyright 2009-2012 by The Regents of the University of California
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
  * you may obtain a copy of the License from
- *
+ * 
  *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.api.dataset;
 
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+package edu.uci.ics.hyracks.storage.am.common.util;
 
-public interface IDatasetPartitionReader {
-    public void writeTo(IFrameWriter writer);
-}
+import java.io.File;
+
+public class IndexFileNameUtil {
+
+    public static String prepareFileName(String path, int ioDeviceId) {
+        return path + File.separator + "device_id_" + ioDeviceId;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
index 06f06c6..eb2e760 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -40,16 +40,17 @@
     public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
     }
 
     @Override
@@ -62,7 +63,8 @@
         return LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ctx.getIOManager(), file, opDesc
                 .getStorageManager().getBufferCache(ctx), opDesc.getStorageManager().getFileMapProvider(ctx),
                 treeOpDesc.getTreeIndexTypeTraits(), treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc
-                        .getTreeIndexBloomFilterKeyFields(), mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider, partition);
+                        .getTreeIndexBloomFilterKeyFields(), bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+                opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
index ebf4bc0..e706786 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
@@ -30,16 +30,17 @@
 
     public LSMBTreeDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
     }
 
     @Override
     public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition) {
         return new LSMBTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
-                mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
-                ioOpCallbackProvider);
+                bloomFilterFalsePositiveRate, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 81c2367..1652f5a 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -93,12 +93,12 @@
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory insertLeafFrameFactory,
             ITreeIndexFrameFactory deleteLeafFrameFactory, ILSMIndexFileManager fileManager,
             TreeIndexFactory<BTree> diskBTreeFactory, TreeIndexFactory<BTree> bulkLoadBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, IFileMapProvider diskFileMapProvider, int fieldCount,
-            IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        super(memFreePageManager, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] cmpFactories,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memFreePageManager, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         mutableComponent = new LSMBTreeMutableComponent(new BTree(memBufferCache,
                 ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), memFreePageManager, interiorFrameFactory,
                 insertLeafFrameFactory, cmpFactories, fieldCount, new FileReference(new File("membtree"))),
@@ -367,7 +367,7 @@
 
         int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
         BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                bloomFilterFalsePositiveRate);
 
         LSMBTreeImmutableComponent component = createDiskComponent(componentFactory, flushOp.getBTreeFlushTarget(),
                 flushOp.getBloomFilterFlushTarget(), true);
@@ -427,7 +427,7 @@
 
         int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
         BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                bloomFilterFalsePositiveRate);
         LSMBTreeImmutableComponent mergedComponent = createDiskComponent(componentFactory,
                 mergeOp.getBTreeMergeTarget(), mergeOp.getBloomFilterMergeTarget(), true);
 
@@ -516,7 +516,7 @@
 
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                    bloomFilterFalsePositiveRate);
             builder = ((LSMBTreeImmutableComponent) component).getBloomFilter().createBuilder(numElementsHint,
                     bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
         }
@@ -647,9 +647,9 @@
             btree.validate();
         }
     }
-    
+
     @Override
     public String toString() {
-        return "LSMBTree [" + fileManager.getBaseDir() + "]"; 
+        return "LSMBTree [" + fileManager.getBaseDir() + "]";
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
index bbcee06..0080c42 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFileManager.java
@@ -27,7 +27,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -41,8 +40,8 @@
     private final TreeIndexFactory<? extends ITreeIndex> btreeFactory;
 
     public LSMBTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> btreeFactory, int startIODeviceIndex) {
-        super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+            TreeIndexFactory<? extends ITreeIndex> btreeFactory, int ioDeviceId) {
+        super(ioManager, fileMapProvider, file, null, ioDeviceId);
         this.btreeFactory = btreeFactory;
     }
 
@@ -80,17 +79,16 @@
         ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from all IODeviceHandles.
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            // List of valid BTree files.
-            cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
-            HashSet<String> btreeFilesSet = new HashSet<String>();
-            for (ComparableFileName cmpFileName : allBTreeFiles) {
-                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
-                btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
-            }
-            validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        // Gather files from the IODeviceHandle.
+
+        // List of valid BTree files.
+        cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+        HashSet<String> btreeFilesSet = new HashSet<String>();
+        for (ComparableFileName cmpFileName : allBTreeFiles) {
+            int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+            btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
         }
+        validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
 
         // Sanity check.
         if (allBTreeFiles.size() != allBloomFilterFiles.size()) {
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
index ac20b6d..d92e93d 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
@@ -45,24 +45,12 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public class LSMBTreeUtils {
-
     public static LSMBTree createLSMTree(IInMemoryBufferCache memBufferCache,
             IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        return createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider,
-                typeTraits, cmpFactories, bloomFilterKeyFields, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider, 0);
-    }
-
-    public static LSMBTree createLSMTree(IInMemoryBufferCache memBufferCache,
-            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int startIODeviceIndex) {
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) {
         LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
                 cmpFactories.length, false);
         LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
@@ -87,12 +75,12 @@
                 bloomFilterKeyFields);
 
         ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, diskFileMapProvider, file,
-                diskBTreeFactory, startIODeviceIndex);
+                diskBTreeFactory, ioDeviceId);
 
         LSMBTree lsmTree = new LSMBTree(memBufferCache, memFreePageManager, interiorFrameFactory,
                 insertLeafFrameFactory, deleteLeafFrameFactory, fileNameManager, diskBTreeFactory,
-                bulkLoadBTreeFactory, bloomFilterFactory, diskFileMapProvider, typeTraits.length, cmpFactories,
-                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                bulkLoadBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider,
+                typeTraits.length, cmpFactories, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return lsmTree;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
index ea7c3b4..6202518 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
@@ -27,9 +27,11 @@
 
     protected static int DEFAULT_MEM_PAGE_SIZE = 32768;
     protected static int DEFAULT_MEM_NUM_PAGES = 1000;
+    protected static double DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE = 0.01;
 
     protected final int memPageSize;
     protected final int memNumPages;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected final ILSMMergePolicy mergePolicy;
     protected final ILSMIOOperationScheduler ioScheduler;
@@ -39,17 +41,19 @@
     public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
         super(opDesc, ctx, partition);
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.mergePolicy = mergePolicy;
         this.opTrackerFactory = opTrackerFactory;
         this.ioScheduler = ioScheduler;
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
index a2f2a11..bc4271c 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
@@ -30,15 +30,18 @@
     protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
     protected final int memPageSize;
     protected final int memNumPages;
+    protected final double bloomFilterFalsePositiveRate;
 
     public AbstractLSMIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         this.mergePolicyProvider = mergePolicyProvider;
         this.opTrackerFactory = opTrackerFactory;
         this.ioSchedulerProvider = ioSchedulerProvider;
         this.ioOpCallbackProvider = ioOpCallbackProvider;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index baa9648..a3e84ed 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -113,7 +113,9 @@
         appender.reset(writeBuffer, true);
         for (int i = startTupleIndex; i < endTupleIndex; i++) {
             if (!appender.append(accessor, i)) {
-                throw new IllegalStateException("Failed to append tuple into frame.");
+                throw new HyracksDataException("Record size ("
+                        + (accessor.getTupleEndOffset(i) - accessor.getTupleStartOffset(i))
+                        + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
             }
         }
         FrameUtils.flushFrame(writeBuffer, writer);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 12c7ea7..6782fbe 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -39,8 +39,6 @@
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
 public abstract class AbstractLSMIndex implements ILSMIndexInternal {
-    protected final static double MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE = 0.01;
-
     protected final ILSMHarness lsmHarness;
 
     protected final ILSMIOOperationScheduler ioScheduler;
@@ -54,19 +52,22 @@
     protected final ILSMIndexFileManager fileManager;
     protected final IFileMapProvider diskFileMapProvider;
     protected final AtomicReference<List<ILSMComponent>> componentsRef;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected boolean isActivated;
 
     private boolean needsFlush = false;
 
     public AbstractLSMIndex(IInMemoryFreePageManager memFreePageManager, IBufferCache diskBufferCache,
-            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ILSMMergePolicy mergePolicy,
+            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider,
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
         this.memFreePageManager = memFreePageManager;
         this.diskBufferCache = diskBufferCache;
         this.diskFileMapProvider = diskFileMapProvider;
         this.fileManager = fileManager;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = ioScheduler;
         this.ioOpCallbackProvider = ioOpCallbackProvider;
         ILSMOperationTracker opTracker = opTrackerFactory.createOperationTracker(this);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
index a808143..99b62d8 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexFileManager.java
@@ -44,10 +44,8 @@
     protected static final String SPLIT_STRING = "_";
     protected static final String BLOOM_FILTER_STRING = "f";
 
-    // Use all IODevices registered in ioManager in a round-robin fashion to choose
-    // where to flush and merge
-    protected final IIOManager ioManager;
     protected final IFileMapProvider fileMapProvider;
+    protected final IODeviceHandle dev;
 
     // baseDir should reflect dataset name and partition name.
     protected String baseDir;
@@ -57,19 +55,15 @@
 
     protected final TreeIndexFactory<? extends ITreeIndex> treeFactory;
 
-    // The current index for the round-robin file assignment
-    private int ioDeviceIndex = 0;
-
     public AbstractLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> treeFactory, int startIODeviceIndex) {
+            TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
         this.baseDir = file.getFile().getPath();
         if (!baseDir.endsWith(System.getProperty("file.separator"))) {
             baseDir += System.getProperty("file.separator");
         }
         this.fileMapProvider = fileMapProvider;
-        this.ioManager = ioManager;
         this.treeFactory = treeFactory;
-        ioDeviceIndex = startIODeviceIndex % ioManager.getIODevices().size();
+        this.dev = ioManager.getIODevices().get(ioDeviceId);
     }
 
     private static FilenameFilter fileNameFilter = new FilenameFilter() {
@@ -135,18 +129,14 @@
 
     @Override
     public void createDirs() {
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            File f = new File(dev.getPath(), baseDir);
-            f.mkdirs();
-        }
+        File f = new File(dev.getPath(), baseDir);
+        f.mkdirs();
     }
 
     @Override
     public void deleteDirs() {
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            File f = new File(dev.getPath(), baseDir);
-            delete(f);
-        }
+        File f = new File(dev.getPath(), baseDir);
+        delete(f);
     }
 
     private void delete(File f) {
@@ -165,9 +155,6 @@
     };
 
     protected FileReference createFlushFile(String relFlushFileName) {
-        // Assigns new files to I/O devices in round-robin fashion.
-        IODeviceHandle dev = ioManager.getIODevices().get(ioDeviceIndex);
-        ioDeviceIndex = (ioDeviceIndex + 1) % ioManager.getIODevices().size();
         return dev.createFileReference(relFlushFileName);
     }
 
@@ -198,14 +185,12 @@
         List<LSMComponentFileReferences> validFiles = new ArrayList<LSMComponentFileReferences>();
         ArrayList<ComparableFileName> allFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from all IODeviceHandles and delete invalid files
+        // Gather files from the IODeviceHandle and delete invalid files
         // There are two types of invalid files:
         // (1) The isValid flag is not set
         // (2) The file's interval is contained by some other file
         // Here, we only filter out (1).
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            cleanupAndGetValidFilesInternal(dev, fileNameFilter, treeFactory, allFiles);
-        }
+        cleanupAndGetValidFilesInternal(dev, fileNameFilter, treeFactory, allFiles);
 
         if (allFiles.isEmpty()) {
             return validFiles;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 6f28f61..5846f1b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -116,7 +116,7 @@
                     FrameUtils.flushFrame(writeBuffer, writer);
                     appender.reset(writeBuffer, true);
                     if (!appender.append(builder.getFieldEndOffsets(), builder.getByteArray(), 0, builder.getSize())) {
-                        throw new IllegalStateException();
+                        throw new HyracksDataException("Record size (" + builder.getSize() +") larger than frame size (" + appender.getBuffer().capacity() + ")");
                     }
                 }
             }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index 3d8b391..c97e10d 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -42,16 +42,17 @@
     public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
             ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
     }
 
     @Override
@@ -66,11 +67,12 @@
             IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
             IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
             LSMInvertedIndex invIndex = InvertedIndexUtils.createLSMInvertedIndex(memBufferCache, memFreePageManager,
-                    diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
-                    invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
-                    invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
-                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), mergePolicy, opTrackerFactory,
-                    ioScheduler, ioOpCallbackProvider, partition);
+                    diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc
+                            .getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(), invIndexOpDesc
+                            .getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(), diskBufferCache, ctx
+                            .getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+                    opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+                    opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
index 9796ebc..60a0b47 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
@@ -30,16 +30,18 @@
 
     public LSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
     }
 
     @Override
     public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition) {
-        return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicyProvider.getMergePolicy(ctx),
-                opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
+        return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
+                bloomFilterFalsePositiveRate, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
     }
 
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index c5b4f07..6c5b25f 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -42,16 +42,17 @@
     public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            int partition, int memPageSize, int memNumPages, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider);
+            int partition, int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
     }
 
     @Override
@@ -66,11 +67,12 @@
             IBufferCache diskBufferCache = opDesc.getStorageManager().getBufferCache(ctx);
             IFileMapProvider diskFileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
             PartitionedLSMInvertedIndex invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(memBufferCache,
-                    memFreePageManager, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(),
-                    invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
-                    invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
-                    diskBufferCache, ctx.getIOManager(), file.getFile().getPath(), mergePolicy, opTrackerFactory,
-                    ioScheduler, ioOpCallbackProvider, partition);
+                    memFreePageManager, diskFileMapProvider, invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc
+                            .getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(), invIndexOpDesc
+                            .getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(), diskBufferCache, ctx
+                            .getIOManager(), file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
+                    opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+                    opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
index 8a8aad2..801462b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
@@ -30,17 +30,17 @@
 
     public PartitionedLSMInvertedIndexDataflowHelperFactory(ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages) {
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerProvider, ioSchedulerProvider, ioOpCallbackProvider, memNumPages,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
     }
 
     @Override
     public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition) {
         return new PartitionedLSMInvertedIndexDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
-                mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
-                ioOpCallbackProvider);
+                bloomFilterFalsePositiveRate, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider);
     }
-
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 30fdd27..beb4dc9 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -97,15 +97,15 @@
 
     public LSMInvertedIndex(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
             OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
-            BloomFilterFactory bloomFilterFactory, ILSMIndexFileManager fileManager,
-            IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+            BloomFilterFactory bloomFilterFactory, double bloomFilterFalsePositiveRate,
+            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
             throws IndexException {
-        super(memFreePageManager, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+        super(memFreePageManager, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         this.memFreePageManager = memFreePageManager;
         this.tokenizerFactory = tokenizerFactory;
         this.invListTypeTraits = invListTypeTraits;
@@ -314,6 +314,7 @@
                 throw new UnsupportedOperationException("Operation " + ctx.getOperation() + " not supported.");
             }
         }
+        mutableComponent.setIsModified();
     }
 
     @Override
@@ -461,7 +462,7 @@
         if (numBTreeTuples > 0) {
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numBTreeTuples);
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                    bloomFilterFalsePositiveRate);
 
             // Create an BTree instance for the deleted keys.
             BTree diskDeletedKeysBTree = component.getDeletedKeysBTree();
@@ -736,9 +737,9 @@
             component.getDeletedKeysBTree().validate();
         }
     }
-    
+
     @Override
     public String toString() {
-        return "LSMInvertedIndex [" + fileManager.getBaseDir() + "]"; 
+        return "LSMInvertedIndex [" + fileManager.getBaseDir() + "]";
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
index ccba624..aa1dbeb 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFileManager.java
@@ -27,7 +27,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
@@ -63,8 +62,8 @@
     };
 
     public LSMInvertedIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            BTreeFactory btreeFactory, int startIODeviceIndex) {
-        super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+            BTreeFactory btreeFactory, int ioDeviceId) {
+        super(ioManager, fileMapProvider, file, null, ioDeviceId);
         this.btreeFactory = btreeFactory;
     }
 
@@ -100,20 +99,19 @@
         ArrayList<ComparableFileName> allDeletedKeysBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from all IODeviceHandles.
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            cleanupAndGetValidFilesInternal(dev, deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
-            HashSet<String> deletedKeysBTreeFilesSet = new HashSet<String>();
-            for (ComparableFileName cmpFileName : allDeletedKeysBTreeFiles) {
-                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
-                deletedKeysBTreeFilesSet.add(cmpFileName.fileName.substring(0, index));
-            }
-
-            // TODO: do we really need to validate the inverted lists files or is validating the dict. BTrees is enough?
-            validateFiles(dev, deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
-            validateFiles(dev, deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
-            validateFiles(dev, deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        // Gather files from the IODeviceHandle.
+        cleanupAndGetValidFilesInternal(dev, deletedKeysBTreeFilter, btreeFactory, allDeletedKeysBTreeFiles);
+        HashSet<String> deletedKeysBTreeFilesSet = new HashSet<String>();
+        for (ComparableFileName cmpFileName : allDeletedKeysBTreeFiles) {
+            int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+            deletedKeysBTreeFilesSet.add(cmpFileName.fileName.substring(0, index));
         }
+
+        // TODO: do we really need to validate the inverted lists files or is validating the dict. BTrees is enough?
+        validateFiles(dev, deletedKeysBTreeFilesSet, allInvListsFiles, invListFilter, null);
+        validateFiles(dev, deletedKeysBTreeFilesSet, allDictBTreeFiles, dictBTreeFilter, btreeFactory);
+        validateFiles(dev, deletedKeysBTreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+
         // Sanity check.
         if (allDictBTreeFiles.size() != allInvListsFiles.size()
                 || allDictBTreeFiles.size() != allDeletedKeysBTreeFiles.size()
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index 1b293eb..a45f729 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -38,15 +38,17 @@
     public PartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
             IInMemoryFreePageManager memFreePageManager, OnDiskInvertedIndexFactory diskInvIndexFactory,
             BTreeFactory deletedKeysBTreeFactory, BloomFilterFactory bloomFilterFactory,
-            ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+            double bloomFilterFalsePositiveRate, ILSMIndexFileManager fileManager,
+            IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
             throws IndexException {
         super(memBufferCache, memFreePageManager, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
-                fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 79c8ccf..06be7aa 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -122,22 +122,9 @@
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
-            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
-            throws IndexException {
-        return createLSMInvertedIndex(memBufferCache, memFreePageManager, diskFileMapProvider, invListTypeTraits,
-                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, diskBufferCache, ioManager,
-                onDiskDir, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, 0);
-    }
-
-    public static LSMInvertedIndex createLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
-            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
-            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            int startIODeviceIndex) throws IndexException {
+            String onDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -151,7 +138,7 @@
 
         FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
         LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
-                onDiskDirFileRef, deletedKeysBTreeFactory, startIODeviceIndex);
+                onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
 
         IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
                 invListTypeTraits);
@@ -160,9 +147,9 @@
                 tokenCmpFactories, fileManager);
 
         LSMInvertedIndex invIndex = new LSMInvertedIndex(memBufferCache, memFreePageManager, invIndexFactory,
-                deletedKeysBTreeFactory, bloomFilterFactory, fileManager, diskFileMapProvider, invListTypeTraits,
-                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager,
+                diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return invIndex;
     }
 
@@ -171,23 +158,9 @@
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
-            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
-            throws IndexException {
-        return createPartitionedLSMInvertedIndex(memBufferCache, memFreePageManager, diskFileMapProvider,
-                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                diskBufferCache, ioManager, onDiskDir, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider, 0);
-    }
-
-    public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
-            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
-            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
-            String onDiskDir, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            int startIODeviceIndex) throws IndexException {
+            String onDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -201,7 +174,7 @@
 
         FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
         LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
-                onDiskDirFileRef, deletedKeysBTreeFactory, startIODeviceIndex);
+                onDiskDirFileRef, deletedKeysBTreeFactory, ioDeviceId);
 
         IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
                 invListTypeTraits);
@@ -210,9 +183,9 @@
                 tokenTypeTraits, tokenCmpFactories, fileManager);
 
         PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
-                invIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory, fileManager, diskFileMapProvider,
-                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
-                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                invIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate,
+                fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, tokenizerFactory, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return invIndex;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
index c363c99..5fcdcea 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -55,19 +55,20 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
             ILinearizeComparatorFactory linearizeCmpFactory) {
-        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES, btreeComparatorFactories,
-                valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider, linearizeCmpFactory);
+        this(opDesc, ctx, partition, DEFAULT_MEM_PAGE_SIZE, DEFAULT_MEM_NUM_PAGES,
+                DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, btreeComparatorFactories, valueProviderFactories,
+                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
     }
 
     public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+            IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
             ILinearizeComparatorFactory linearizeCmpFactory) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, mergePolicy,
+                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         this.btreeComparatorFactories = btreeComparatorFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index 1df914e..5ab5513 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -50,13 +50,15 @@
     }
 
     public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            int memPageSize, int memNumPages, IBinaryComparatorFactory[] btreeComparatorFactories,
+            int memPageSize, int memNumPages, double bloomFilterFalsePositiveRate,
+            IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
             ILinearizeComparatorFactory linearizeCmpFactory) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, btreeComparatorFactories, valueProviderFactories,
-                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+        super(opDesc, ctx, partition, memPageSize, memNumPages, bloomFilterFalsePositiveRate, btreeComparatorFactories,
+                valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider, linearizeCmpFactory);
     }
 
     @Override
@@ -69,8 +71,9 @@
         try {
             return LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache,
                     diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
-                    rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
-                    linearizeCmpFactory, startIODeviceIndex);
+                    rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
+                    ioOpCallbackProvider, linearizeCmpFactory,
+                    opDesc.getFileSplitProvider().getFileSplits()[partition].getIODeviceId());
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
index a730895..0778e15 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -41,9 +41,10 @@
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
             ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize, int memNumPages) {
+            ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize, int memNumPages,
+            double bloomFilterFalsePositiveRate) {
         super(mergePolicyProvider, opTrackerFactory, ioSchedulerProvider, ioOpCallbackProvider, memPageSize,
-                memNumPages);
+                memNumPages, bloomFilterFalsePositiveRate);
         this.btreeComparatorFactories = btreeComparatorFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -53,8 +54,9 @@
     @Override
     public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition) {
-        return new LSMRTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages, btreeComparatorFactories, valueProviderFactories,
-                rtreePolicyType, mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory,
-                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider, linearizeCmpFactory);
+        return new LSMRTreeDataflowHelper(opDesc, ctx, partition, memPageSize, memNumPages,
+                bloomFilterFalsePositiveRate, btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
+                mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
+                ioOpCallbackProvider, linearizeCmpFactory);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
index 6f5ecb1..d4fac8f 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -54,7 +54,7 @@
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
             ILinearizeComparatorFactory linearizeCmpFactory) {
-        super(opDesc, ctx, partition, memPageSize, memNumPages, btreeComparatorFactories, valueProviderFactories,
+        super(opDesc, ctx, partition, memPageSize, memNumPages, 0, btreeComparatorFactories, valueProviderFactories,
                 rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index 4b2e075..f62fab1 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -91,10 +91,11 @@
             ILSMComponentFactory componentFactory, IFileMapProvider diskFileMapProvider, int fieldCount,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
             ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
-        super(memFreePageManager, diskRTreeFactory.getBufferCache(), fileManager, diskFileMapProvider, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+        super(memFreePageManager, diskRTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         RTree memRTree = new RTree(memBufferCache, ((InMemoryBufferCache) memBufferCache).getFileMapProvider(),
                 memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, rtreeCmpFactories, fieldCount,
                 new FileReference(new File("memrtree")));
@@ -331,7 +332,7 @@
                 NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         BTreeAccessor btreeAccessor = (BTree.BTreeAccessor) mutableComponent.getBTree().createAccessor(
                 NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        
+
         return new LSMRTreeOpContext(rtreeAccessor, (IRTreeLeafFrame) rtreeLeafFrameFactory.createFrame(),
                 (IRTreeInteriorFrame) rtreeInteriorFrameFactory.createFrame(), memFreePageManager
                         .getMetaDataFrameFactory().createFrame(), 4, btreeAccessor, btreeLeafFrameFactory,
@@ -362,9 +363,9 @@
         InMemoryBufferCache memBufferCache = (InMemoryBufferCache) mutableComponent.getRTree().getBufferCache();
         return memBufferCache.getNumPages() * memBufferCache.getPageSize();
     }
-    
+
     @Override
     public String toString() {
-        return "LSMRTree [" + fileManager.getBaseDir() + "]"; 
+        return "LSMRTree [" + fileManager.getBaseDir() + "]";
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index fc5b06d..367b387 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -72,16 +72,17 @@
             ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
             ILSMIndexFileManager fileNameManager, TreeIndexFactory<RTree> diskRTreeFactory,
             TreeIndexFactory<BTree> diskBTreeFactory, BloomFilterFactory bloomFilterFactory,
-            IFileMapProvider diskFileMapProvider, int fieldCount, IBinaryComparatorFactory[] rtreeCmpFactories,
-            IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
-            int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount,
+            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
+            ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
         super(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
                 btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory,
                 new LSMRTreeComponentFactory(diskRTreeFactory, diskBTreeFactory, bloomFilterFactory),
                 diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
-                linearizerArray, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                linearizerArray, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler,
+                ioOpCallbackProvider);
     }
 
     /**
@@ -303,7 +304,7 @@
         if (numBTreeTuples > 0) {
             int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numBTreeTuples);
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
-                    MAX_BLOOM_FILTER_ACCEPTABLE_FALSE_POSITIVE_RATE);
+                    bloomFilterFalsePositiveRate);
 
             IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor();
             memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
index 851235e..fd36920 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFileManager.java
@@ -27,7 +27,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -56,8 +55,8 @@
 
     public LSMRTreeFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
             TreeIndexFactory<? extends ITreeIndex> rtreeFactory, TreeIndexFactory<? extends ITreeIndex> btreeFactory,
-            int startIODeviceIndex) {
-        super(ioManager, fileMapProvider, file, null, startIODeviceIndex);
+            int ioDeviceId) {
+        super(ioManager, fileMapProvider, file, null, ioDeviceId);
         this.rtreeFactory = rtreeFactory;
         this.btreeFactory = btreeFactory;
     }
@@ -93,17 +92,16 @@
         ArrayList<ComparableFileName> allBTreeFiles = new ArrayList<ComparableFileName>();
         ArrayList<ComparableFileName> allBloomFilterFiles = new ArrayList<ComparableFileName>();
 
-        // Gather files from all IODeviceHandles.
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
-            HashSet<String> btreeFilesSet = new HashSet<String>();
-            for (ComparableFileName cmpFileName : allBTreeFiles) {
-                int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
-                btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
-            }
-            validateFiles(dev, btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
-            validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+        // Gather files from the IODeviceHandle.
+        cleanupAndGetValidFilesInternal(dev, btreeFilter, btreeFactory, allBTreeFiles);
+        HashSet<String> btreeFilesSet = new HashSet<String>();
+        for (ComparableFileName cmpFileName : allBTreeFiles) {
+            int index = cmpFileName.fileName.lastIndexOf(SPLIT_STRING);
+            btreeFilesSet.add(cmpFileName.fileName.substring(0, index));
         }
+        validateFiles(dev, btreeFilesSet, allRTreeFiles, rtreeFilter, rtreeFactory);
+        validateFiles(dev, btreeFilesSet, allBloomFilterFiles, bloomFilterFilter, null);
+
         // Sanity check.
         if (allRTreeFiles.size() != allBTreeFiles.size() || allBTreeFiles.size() != allBloomFilterFiles.size()) {
             throw new HyracksDataException(
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index be9073e..a3a8ca2 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -81,7 +81,7 @@
         super(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory, rtreeLeafFrameFactory,
                 btreeInteriorFrameFactory, btreeLeafFrameFactory, fileManager, diskRTreeFactory,
                 new LSMRTreeWithAntiMatterTuplesComponentFactory(diskRTreeFactory), diskFileMapProvider, fieldCount,
-                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, mergePolicy,
+                rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields, linearizerArray, 0, mergePolicy,
                 opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         bulkLoaComponentFactory = new LSMRTreeWithAntiMatterTuplesComponentFactory(bulkLoadRTreeFactory);
         this.bTreeTupleSorter = null;
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
index 6ddf766..872d152 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFileManager.java
@@ -25,7 +25,7 @@
 public class LSMRTreeWithAntiMatterTuplesFileManager extends AbstractLSMIndexFileManager {
 
     public LSMRTreeWithAntiMatterTuplesFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider,
-            FileReference file, TreeIndexFactory<? extends ITreeIndex> rtreeFactory, int startIODeviceIndex) {
-        super(ioManager, fileMapProvider, file, rtreeFactory, startIODeviceIndex);
+            FileReference file, TreeIndexFactory<? extends ITreeIndex> rtreeFactory, int ioDeviceId) {
+        super(ioManager, fileMapProvider, file, rtreeFactory, ioDeviceId);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index 6c9fce6..d299bfe 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -65,22 +65,10 @@
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            ILinearizeComparatorFactory linearizeCmpFactory) throws TreeIndexException {
-        return createLSMTree(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider,
-                typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory, 0);
-    }
-
-    public static LSMRTree createLSMTree(IInMemoryBufferCache memBufferCache,
-            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            ILinearizeComparatorFactory linearizeCmpFactory, int startIODeviceIndex) throws TreeIndexException {
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
+            int ioDeviceId) throws TreeIndexException {
         LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
         LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, true);
 
@@ -114,12 +102,12 @@
                 bloomFilterKeyFields);
 
         ILSMIndexFileManager fileNameManager = new LSMRTreeFileManager(ioManager, diskFileMapProvider, file,
-                diskRTreeFactory, diskBTreeFactory, startIODeviceIndex);
+                diskRTreeFactory, diskBTreeFactory, ioDeviceId);
         LSMRTree lsmTree = new LSMRTree(memBufferCache, memFreePageManager, rtreeInteriorFrameFactory,
                 rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager,
-                diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, diskFileMapProvider, typeTraits.length,
-                rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
-                mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                diskRTreeFactory, diskBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate,
+                diskFileMapProvider, typeTraits.length, rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory,
+                comparatorFields, linearizerArray, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider);
         return lsmTree;
     }
 
@@ -130,21 +118,7 @@
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            ILinearizeComparatorFactory linearizerCmpFactory) throws TreeIndexException {
-        return createLSMTreeWithAntiMatterTuples(memBufferCache, memFreePageManager, ioManager, file, diskBufferCache,
-                diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
-                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
-                linearizerCmpFactory, 0);
-    }
-
-    public static LSMRTreeWithAntiMatterTuples createLSMTreeWithAntiMatterTuples(IInMemoryBufferCache memBufferCache,
-            IInMemoryFreePageManager memFreePageManager, IIOManager ioManager, FileReference file,
-            IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
-            ILinearizeComparatorFactory linearizerCmpFactory, int startIODeviceIndex) throws TreeIndexException {
+            ILinearizeComparatorFactory linearizerCmpFactory, int ioDeviceId) throws TreeIndexException {
 
         LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
         LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
@@ -181,7 +155,7 @@
                 btreeCmpFactories[btreeCmpFactories.length - 1] };
 
         ILSMIndexFileManager fileNameManager = new LSMRTreeWithAntiMatterTuplesFileManager(ioManager,
-                diskFileMapProvider, file, diskRTreeFactory, startIODeviceIndex);
+                diskFileMapProvider, file, diskRTreeFactory, ioDeviceId);
         LSMRTreeWithAntiMatterTuples lsmTree = new LSMRTreeWithAntiMatterTuples(memBufferCache, memFreePageManager,
                 rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
                 fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
index 5ab9632..f197853 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
@@ -48,6 +48,11 @@
     }
 
     @Override
+    public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+        return tupleWriter.bytesRequired(tuple) + childPtrSize + slotManager.getSlotSize();
+    }
+
+    @Override
     public int findBestChild(ITupleReference tuple, MultiComparator cmp) {
         int bestChild = rtreePolicy.findBestChildPosition(this, tuple, frameTuple, cmp);
         frameTuple.resetByTupleIndex(this, bestChild);
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
index d52ef16..4057778 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMLeafFrame.java
@@ -30,6 +30,11 @@
     }
 
     @Override
+    public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
+        return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize();
+    }
+
+    @Override
     public ITreeIndexTupleReference createTupleReference() {
         return tupleWriter.createTupleReference();
     }
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index c12dc50..48fb6f1 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -61,12 +61,18 @@
     // Global node sequence number used for the concurrency control protocol
     private final AtomicLong globalNsn;
 
+    private final int maxTupleSize;
+
     public RTree(IBufferCache bufferCache, IFileMapProvider fileMapProvider, IFreePageManager freePageManager,
             ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
             IBinaryComparatorFactory[] cmpFactories, int fieldCount, FileReference file) {
         super(bufferCache, fileMapProvider, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
                 fieldCount, file);
         globalNsn = new AtomicLong();
+        ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+        ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+        maxTupleSize = Math.min(leafFrame.getMaxTupleSize(bufferCache.getPageSize()),
+                interiorFrame.getMaxTupleSize(bufferCache.getPageSize()));
     }
 
     private long incrementGlobalNsn() {
@@ -147,6 +153,12 @@
     private void insert(ITupleReference tuple, IIndexOperationContext ictx) throws HyracksDataException,
             TreeIndexException {
         RTreeOpContext ctx = (RTreeOpContext) ictx;
+        int tupleSize = Math.max(ctx.leafFrame.getBytesRequriedToWriteTuple(tuple),
+                ctx.interiorFrame.getBytesRequriedToWriteTuple(tuple));
+        if (tupleSize > maxTupleSize) {
+            throw new TreeIndexException("Record size (" + tupleSize + ") larger than maximum acceptable record size ("
+                    + maxTupleSize + ")");
+        }
         ctx.reset();
         ctx.setTuple(tuple);
         ctx.splitKey.reset();
@@ -614,7 +626,7 @@
         ctx.splitKey.reset();
         ctx.splitKey.getLeftTuple().setFieldCount(cmpFactories.length);
 
-        // We delete the first matching tuple (including the payload data.
+        // We delete the first matching tuple (including the payload data).
         // We don't update the MBRs of the parents after deleting the record.
         int tupleIndex = findTupleToDelete(ctx);
 
@@ -870,8 +882,15 @@
         }
 
         @Override
-        public void add(ITupleReference tuple) throws HyracksDataException {
+        public void add(ITupleReference tuple) throws IndexException, HyracksDataException {
             try {
+                int tupleSize = Math.max(leafFrame.getBytesRequriedToWriteTuple(tuple),
+                        interiorFrame.getBytesRequriedToWriteTuple(tuple));
+                if (tupleSize > maxTupleSize) {
+                    throw new TreeIndexException("Space required for record (" + tupleSize
+                            + ") larger than maximum acceptable size (" + maxTupleSize + ")");
+                }
+
                 NodeFrontier leafFrontier = nodeFrontiers.get(0);
 
                 int spaceNeeded = tupleWriter.bytesRequired(tuple) + slotSize;
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
index ab9ec41..36aa088 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/ILocalResourceRepository.java
@@ -19,16 +19,16 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface ILocalResourceRepository {
-	
+
     public LocalResource getResourceById(long id) throws HyracksDataException;
 
     public LocalResource getResourceByName(String name) throws HyracksDataException;
 
-    public void insert(LocalResource resource) throws HyracksDataException;
+    public void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException;
 
-    public void deleteResourceById(long id) throws HyracksDataException;
+    public void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException;
 
-    public void deleteResourceByName(String name) throws HyracksDataException;
+    public void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException;
 
     public List<LocalResource> getAllResources() throws HyracksDataException;
 }
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
index 55bd807..f853a36 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/TransientLocalResourceRepository.java
@@ -37,7 +37,7 @@
     }
 
     @Override
-    public synchronized void insert(LocalResource resource) throws HyracksDataException {
+    public synchronized void insert(LocalResource resource, int ioDeviceId) throws HyracksDataException {
         long id = resource.getResourceId();
 
         if (id2ResourceMap.containsKey(id)) {
@@ -48,7 +48,7 @@
     }
 
     @Override
-    public synchronized void deleteResourceById(long id) throws HyracksDataException {
+    public synchronized void deleteResourceById(long id, int ioDeviceId) throws HyracksDataException {
         LocalResource resource = id2ResourceMap.get(id);
         if (resource == null) {
             throw new HyracksDataException("Resource doesn't exist");
@@ -58,7 +58,7 @@
     }
 
     @Override
-    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
+    public synchronized void deleteResourceByName(String name, int ioDeviceId) throws HyracksDataException {
         LocalResource resource = name2ResourceMap.get(name);
         if (resource == null) {
             throw new HyracksDataException("Resource doesn't exist");
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
index f962200..73176f3 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
@@ -53,6 +53,7 @@
     public static final int LSM_RTREE_MEM_PAGE_SIZE = 512;
     public static final int LSM_RTREE_MEM_NUM_PAGES = 1000;
     public static final int LSM_RTREE_HYRACKS_FRAME_SIZE = 128;
+    public static final double LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
 
     // Mem configuration for BTree.
     public static final int BTREE_PAGE_SIZE = 256;
@@ -67,6 +68,7 @@
     public static final int LSM_BTREE_MEM_PAGE_SIZE = 256;
     public static final int LSM_BTREE_MEM_NUM_PAGES = 100;
     public static final int LSM_BTREE_HYRACKS_FRAME_SIZE = 128;
+    public static final double LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
 
     // Mem configuration for Inverted Index.
     public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
@@ -75,6 +77,7 @@
     public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
     public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
     public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+    public static final double LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE = 0.01;
     // Test parameters.
     public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 100;
     // Used for full-fledged search test.
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index f93e9b6..fdb4341 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -225,7 +225,10 @@
         int p1y = rnd.nextInt();
         int p2x = rnd.nextInt();
         int p2y = rnd.nextInt();
-        String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        String data = "";
+        for (int i = 0; i < 210; i++) {
+            data += "X";
+        }
         TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                 Math.max(p1y, p2y), data);
         indexAccessor.insert(tuple);
@@ -261,7 +264,10 @@
         p1y = rnd.nextInt();
         p2x = rnd.nextInt();
         p2y = rnd.nextInt();
-        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        data = "";
+        for (int i = 0; i < 210; i++) {
+            data += "X";
+        }
         TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                 Math.max(p1y, p2y), data);
         indexAccessor.insert(tuple);
@@ -270,7 +276,10 @@
         p1y = rnd.nextInt();
         p2x = rnd.nextInt();
         p2y = rnd.nextInt();
-        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        data = "";
+        for (int i = 0; i < 210; i++) {
+            data += "X";
+        }
         TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                 Math.max(p1y, p2y), data);
         indexAccessor.insert(tuple);
@@ -340,7 +349,10 @@
         int p1y = rnd.nextInt();
         int p2x = rnd.nextInt();
         int p2y = rnd.nextInt();
-        String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        String data = "";
+        for (int i = 0; i < 210; i++) {
+            data += "X";
+        }
         TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                 Math.max(p1y, p2y), data);
         indexAccessor.insert(tuple);
@@ -349,7 +361,10 @@
         p1y = rnd.nextInt();
         p2x = rnd.nextInt();
         p2y = rnd.nextInt();
-        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        data = "";
+        for (int i = 0; i < 210; i++) {
+            data += "X";
+        }
         TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                 Math.max(p1y, p2y), data);
         indexAccessor.insert(tuple);
@@ -376,7 +391,10 @@
         p1y = rnd.nextInt();
         p2x = rnd.nextInt();
         p2y = rnd.nextInt();
-        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        data = "";
+        for (int i = 0; i < 210; i++) {
+            data += "X";
+        }
         TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                 Math.max(p1y, p2y), data);
         indexAccessor.insert(tuple);
@@ -385,7 +403,10 @@
         p1y = rnd.nextInt();
         p2x = rnd.nextInt();
         p2y = rnd.nextInt();
-        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        data = "";
+        for (int i = 0; i < 210; i++) {
+            data += "X";
+        }
         TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                 Math.max(p1y, p2y), data);
         indexAccessor.insert(tuple);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index 4bd1910..a7eee57 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -53,9 +53,9 @@
             BTreeLeafFrameType leafType) throws Exception {
         return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),harness.getMergePolicy(),
                 harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index 069faad..a648ff6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -53,9 +53,9 @@
             BTreeLeafFrameType leafType) throws Exception {
         return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
-                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index 539ed3e..b5b0b76 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -37,8 +37,9 @@
         return LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), typeTraits, cmpFactories, bloomFilterKeyFields,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index f17e3c8..875bbd4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -53,9 +53,9 @@
             BTreeLeafFrameType leafType) throws Exception {
         return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
-                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index 24d1f10..43d38bc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -44,9 +44,10 @@
         harness.setUp();
         testCtx = LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length, harness.getMergePolicy(),
+                harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index da36c79..0ea2af2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -52,9 +52,9 @@
             BTreeLeafFrameType leafType) throws Exception {
         return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
-                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 648e70f..901a31d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -46,8 +46,9 @@
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
-                harness.getMergePolicy(), NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index 3a99c16..ac8290e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -54,9 +54,9 @@
             BTreeLeafFrameType leafType) throws Exception {
         return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
-                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index 41d7ae8..33e1641 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -39,8 +39,9 @@
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), SerdeUtils.serdesToTypeTraits(keySerdes),
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
-                harness.getMergePolicy(), NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                NoOpOperationTrackerFactory.INSTANCE, harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index ca89512..cc2db11 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -53,9 +53,9 @@
             BTreeLeafFrameType leafType) throws Exception {
         return LSMBTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
-                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getMergePolicy(),
-                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getDiskFileMapProvider(), fieldSerdes, numKeys, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index c494448..d56ff5b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -53,8 +53,9 @@
         return LSMBTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), typeTraits, cmpFactories, bloomFilterKeyFields,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 5d2185a..7e54003 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -57,6 +57,7 @@
 
     protected IHyracksTaskContext ctx;
     protected IOManager ioManager;
+    protected int ioDeviceId;
     protected IBufferCache bufferCache;
     protected int lsmtreeFileId;
 
@@ -74,8 +75,8 @@
     private final int onDiskNumPages;
 
     public LSMTreeRunner(int numBatches, int inMemPageSize, int inMemNumPages, int onDiskPageSize, int onDiskNumPages,
-            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields)
-            throws BTreeException, HyracksException {
+            ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
+            double bloomFilterFalsePositiveRate) throws BTreeException, HyracksException {
         this.numBatches = numBatches;
 
         this.onDiskPageSize = onDiskPageSize;
@@ -88,6 +89,7 @@
         TestStorageManagerComponentHolder.init(this.onDiskPageSize, this.onDiskNumPages, MAX_OPEN_FILES);
         bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
         ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ioDeviceId = 0;
         IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
 
         IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), inMemPageSize,
@@ -96,8 +98,9 @@
                 new LIFOMetaDataFrameFactory());
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         lsmtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, bufferCache, fmp,
-                typeTraits, cmpFactories, bloomFilterKeyFields, NoMergePolicy.INSTANCE,
-                ThreadCountingOperationTrackerFactory.INSTANCE, ioScheduler, NoOpIOOperationCallback.INSTANCE);
+                typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
+                ThreadCountingOperationTrackerFactory.INSTANCE, ioScheduler, NoOpIOOperationCallback.INSTANCE,
+                ioDeviceId);
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index f790fde..a830dc0 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -66,9 +66,9 @@
     public static LSMBTreeTestContext create(IInMemoryBufferCache memBufferCache,
             IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
-            int numKeyFields, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
-            throws Exception {
+            int numKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
         int[] bloomFilterKeyFields = new int[numKeyFields];
@@ -76,8 +76,9 @@
             bloomFilterKeyFields[i] = i;
         }
         LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
-                diskBufferCache, diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                diskBufferCache, diskFileMapProvider, typeTraits, cmpFactories, bloomFilterKeyFields,
+                bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider,
+                ioDeviceId);
         LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 9128607..066f57b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -63,8 +63,10 @@
     protected final int memPageSize;
     protected final int memNumPages;
     protected final int hyracksFrameSize;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected IOManager ioManager;
+    protected int ioDeviceId;
     protected IBufferCache diskBufferCache;
     protected IFileMapProvider diskFileMapProvider;
     protected IInMemoryBufferCache memBufferCache;
@@ -88,6 +90,7 @@
         this.memPageSize = AccessMethodTestsConfig.LSM_BTREE_MEM_PAGE_SIZE;
         this.memNumPages = AccessMethodTestsConfig.LSM_BTREE_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_BTREE_HYRACKS_FRAME_SIZE;
+        this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -95,13 +98,14 @@
     }
 
     public LSMBTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
-            int memNumPages, int hyracksFrameSize) {
+            int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
         this.diskPageSize = diskPageSize;
         this.diskNumPages = diskNumPages;
         this.diskMaxOpenFiles = diskMaxOpenFiles;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
         this.hyracksFrameSize = hyracksFrameSize;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -118,6 +122,7 @@
                 new TransientFileMapManager());
         memFreePageManager = new InMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
         ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ioDeviceId = 0;
         rnd.setSeed(RANDOM_SEED);
     }
 
@@ -169,6 +174,10 @@
         return ioManager;
     }
 
+    public int getIODeviceId() {
+        return ioDeviceId;
+    }
+
     public IBufferCache getDiskBufferCache() {
         return diskBufferCache;
     }
@@ -181,6 +190,10 @@
         return memBufferCache;
     }
 
+    public double getBoomFilterFalsePositiveRate() {
+        return bloomFilterFalsePositiveRate;
+    }
+
     public IInMemoryFreePageManager getMemFreePageManager() {
         return memFreePageManager;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
index 69e23bc..d41ddd4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/DummyLSMIndexFileManager.java
@@ -32,8 +32,8 @@
 public class DummyLSMIndexFileManager extends AbstractLSMIndexFileManager {
 
     public DummyLSMIndexFileManager(IIOManager ioManager, IFileMapProvider fileMapProvider, FileReference file,
-            TreeIndexFactory<? extends ITreeIndex> treeFactory) {
-        super(ioManager, fileMapProvider, file, treeFactory, 0);
+            TreeIndexFactory<? extends ITreeIndex> treeFactory, int ioDeviceId) {
+        super(ioManager, fileMapProvider, file, treeFactory, ioDeviceId);
     }
 
     protected void cleanupAndGetValidFilesInternal(IODeviceHandle dev, FilenameFilter filter,
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
index 161f4ce..0c9e658 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/common/LSMIndexFileManagerTest.java
@@ -48,6 +48,7 @@
     private static final int DEFAULT_PAGE_SIZE = 256;
     private static final int DEFAULT_NUM_PAGES = 100;
     private static final int DEFAULT_MAX_OPEN_FILES = 10;
+    private static final int DEFAULT_IO_DEVICE_ID = 0;
     protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
     protected final static String sep = System.getProperty("file.separator");
     protected IOManager ioManager;
@@ -74,7 +75,7 @@
 
     public void sortOrderTest(boolean testFlushFileName) throws InterruptedException, HyracksDataException {
         ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
-                new DummyTreeFactory());
+                new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
         LinkedList<String> fileNames = new LinkedList<String>();
 
         int numFileNames = 100;
@@ -115,7 +116,7 @@
 
     public void cleanInvalidFilesTest(IOManager ioManager) throws InterruptedException, IOException, IndexException {
         ILSMIndexFileManager fileManager = new DummyLSMIndexFileManager(ioManager, fileMapProvider, file,
-                new DummyTreeFactory());
+                new DummyTreeFactory(), DEFAULT_IO_DEVICE_ID);
         fileManager.createDirs();
 
         List<FileReference> flushFiles = new ArrayList<FileReference>();
@@ -187,20 +188,18 @@
                     .getFile().getName());
         }
 
-        // Make sure invalid files were removed from all IODevices.
+        // Make sure invalid files were removed from the IODevices.
         ArrayList<String> remainingFiles = new ArrayList<String>();
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            File dir = new File(dev.getPath(), baseDir);
-            FilenameFilter filter = new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    return !name.startsWith(".");
-                }
-            };
-            String[] files = dir.list(filter);
-            for (String file : files) {
-                File f = new File(file);
-                remainingFiles.add(f.getName());
+        File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+        FilenameFilter filter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                return !name.startsWith(".");
             }
+        };
+        String[] files = dir.list(filter);
+        for (String file : files) {
+            File f = new File(file);
+            remainingFiles.add(f.getName());
         }
 
         Collections.sort(remainingFiles, fileManager.getFileNameComparator());
@@ -218,34 +217,19 @@
         cleanDirs(singleDeviceIOManager);
     }
 
-    @Test
-    public void twoIODevicesTest() throws InterruptedException, IOException, IndexException {
-        IOManager twoDevicesIOManager = createIOManager(2);
-        cleanInvalidFilesTest(twoDevicesIOManager);
-        cleanDirs(twoDevicesIOManager);
-    }
-
-    @Test
-    public void fourIODevicesTest() throws InterruptedException, IOException, IndexException {
-        IOManager fourDevicesIOManager = createIOManager(4);
-        cleanInvalidFilesTest(fourDevicesIOManager);
-        cleanDirs(fourDevicesIOManager);
-    }
-
     private void cleanDirs(IOManager ioManager) {
-        for (IODeviceHandle dev : ioManager.getIODevices()) {
-            File dir = new File(dev.getPath(), baseDir);
-            FilenameFilter filter = new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    return !name.startsWith(".");
-                }
-            };
-            String[] files = dir.list(filter);
-            for (String file : files) {
-                File f = new File(file);
-                f.delete();
+        File dir = new File(ioManager.getIODevices().get(DEFAULT_IO_DEVICE_ID).getPath(), baseDir);
+        FilenameFilter filter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                return !name.startsWith(".");
             }
+        };
+        String[] files = dir.list(filter);
+        for (String file : files) {
+            File f = new File(file);
+            f.delete();
         }
+
     }
 
     private IOManager createIOManager(int numDevices) throws HyracksException {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 5be1d6a..bf2d5e8 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -57,8 +57,10 @@
     protected final int memPageSize;
     protected final int memNumPages;
     protected final int hyracksFrameSize;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected IOManager ioManager;
+    protected int ioDeviceId;
     protected IBufferCache diskBufferCache;
     protected IFileMapProvider diskFileMapProvider;
     protected IInMemoryBufferCache memBufferCache;
@@ -84,6 +86,7 @@
         this.memPageSize = AccessMethodTestsConfig.LSM_INVINDEX_MEM_PAGE_SIZE;
         this.memNumPages = AccessMethodTestsConfig.LSM_INVINDEX_MEM_NUM_PAGES;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_INVINDEX_HYRACKS_FRAME_SIZE;
+        this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -91,13 +94,14 @@
     }
 
     public LSMInvertedIndexTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
-            int memNumPages, int hyracksFrameSize) {
+            int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
         this.diskPageSize = diskPageSize;
         this.diskNumPages = diskNumPages;
         this.diskMaxOpenFiles = diskMaxOpenFiles;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
         this.hyracksFrameSize = hyracksFrameSize;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTrackerFactory = ThreadCountingOperationTrackerFactory.INSTANCE;
@@ -113,6 +117,7 @@
         memBufferCache.open();
         memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
         ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ioDeviceId = 0;
         rnd.setSeed(RANDOM_SEED);
         invIndexFileRef = ioManager.getIODevices().get(0).createFileReference(onDiskDir + invIndexFileName);
     }
@@ -170,6 +175,10 @@
         return ioManager;
     }
 
+    public int getIODeviceId() {
+        return ioDeviceId;
+    }
+
     public IBufferCache getDiskBufferCache() {
         return diskBufferCache;
     }
@@ -182,6 +191,10 @@
         return memBufferCache;
     }
 
+    public double getBoomFilterFalsePositiveRate() {
+        return bloomFilterFalsePositiveRate;
+    }
+
     public IInMemoryFreePageManager getMemFreePageManager() {
         return memFreePageManager;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 870e6d9..f955fc9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -146,8 +146,9 @@
                         harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                         harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
-                        harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackProvider());
+                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                        harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                        harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
                 break;
             }
             case PARTITIONED_LSM: {
@@ -155,8 +156,9 @@
                         harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
                         harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
-                        harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackProvider());
+                        harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                        harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                        harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
                 break;
             }
             default: {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index 995f18c..edd24b4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -57,8 +57,9 @@
         return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index d72b668..360c02c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -57,8 +57,9 @@
         return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index cba8cee..0c68fa6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -40,10 +40,10 @@
         return LSMRTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
-                valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
     }
 
     @Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index 96485f8..f39b139 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -57,8 +57,9 @@
         return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index e72b3ca..8bee460 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -55,8 +55,9 @@
         testCtx = LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 1d07484..5d7d31b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -56,8 +56,9 @@
         return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
index cc46065..8ef0a9f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMultiBulkLoadTest.java
@@ -57,8 +57,9 @@
         return LSMRTreeTestContext.create(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType,
-                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
index 81a952d..f778aa9 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -58,7 +58,7 @@
                 harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
 
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 1ee92d9..44d3d1b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -58,7 +58,7 @@
                 harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
 
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 3a2537c..8b12224 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -43,7 +43,7 @@
                 btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
                 harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
     }
 
     @Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index 61d5ce7..96ef868 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -58,7 +58,7 @@
                 harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
 
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
index aee8670..012559d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -56,7 +56,7 @@
                 harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
index d5fecbf..1c67cd0 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -57,7 +57,7 @@
                 harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
index de5f065..fe61afd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMultiBulkLoadTest.java
@@ -58,7 +58,7 @@
                 harness.getMemFreePageManager(), harness.getIOManager(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getMergePolicy(), harness.getOperationTrackerFactory(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallbackProvider(), harness.getIODeviceId());
 
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 37be58d..1d2271c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -60,10 +60,10 @@
         return LSMRTreeUtils.createLSMTree(harness.getMemBufferCache(), harness.getMemFreePageManager(),
                 harness.getIOManager(), harness.getFileReference(), harness.getDiskBufferCache(),
                 harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
-                valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTrackerFactory(), harness.getIOScheduler(),
+                valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
+                harness.getMergePolicy(), harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index af73676..aa5023d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -63,7 +63,7 @@
                 btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getMergePolicy(),
                 harness.getOperationTrackerFactory(), harness.getIOScheduler(),
                 harness.getIOOperationCallbackProvider(),
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), harness.getIODeviceId());
 
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index 27fb9c8..fb9fb23 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -71,9 +71,9 @@
             IInMemoryFreePageManager memFreePageManager, IOManager ioManager, FileReference file,
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
-            ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
-            throws Exception {
+            double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
+            ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
                 .serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -81,8 +81,9 @@
                 fieldSerdes.length);
         LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
                 diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
-                valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider, LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+                valueProviderFactories, rtreePolicyType, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
+                ioScheduler, ioOpCallbackProvider,
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), ioDeviceId);
         LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 6fb6f9c..4c2e83b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -59,8 +59,10 @@
     protected final int memPageSize;
     protected final int memNumPages;
     protected final int hyracksFrameSize;
+    protected final double bloomFilterFalsePositiveRate;
 
     protected IOManager ioManager;
+    protected int ioDeviceId;
     protected IBufferCache diskBufferCache;
     protected IFileMapProvider diskFileMapProvider;
     protected IInMemoryBufferCache memBufferCache;
@@ -83,6 +85,7 @@
         this.diskMaxOpenFiles = AccessMethodTestsConfig.LSM_RTREE_DISK_MAX_OPEN_FILES;
         this.memPageSize = AccessMethodTestsConfig.LSM_RTREE_MEM_PAGE_SIZE;
         this.memNumPages = AccessMethodTestsConfig.LSM_RTREE_MEM_NUM_PAGES;
+        this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
         this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
@@ -91,12 +94,13 @@
     }
 
     public LSMRTreeTestHarness(int diskPageSize, int diskNumPages, int diskMaxOpenFiles, int memPageSize,
-            int memNumPages, int hyracksFrameSize) {
+            int memNumPages, int hyracksFrameSize, double bloomFilterFalsePositiveRate) {
         this.diskPageSize = diskPageSize;
         this.diskNumPages = diskNumPages;
         this.diskMaxOpenFiles = diskMaxOpenFiles;
         this.memPageSize = memPageSize;
         this.memNumPages = memNumPages;
+        this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.hyracksFrameSize = hyracksFrameSize;
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
@@ -113,6 +117,7 @@
         memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize, memNumPages);
         memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages, new LIFOMetaDataFrameFactory());
         ioManager = TestStorageManagerComponentHolder.getIOManager();
+        ioDeviceId = 0;
         rnd.setSeed(RANDOM_SEED);
     }
 
@@ -164,6 +169,10 @@
         return ioManager;
     }
 
+    public int getIODeviceId() {
+        return ioDeviceId;
+    }
+
     public IBufferCache getDiskBufferCache() {
         return diskBufferCache;
     }
@@ -176,6 +185,10 @@
         return memBufferCache;
     }
 
+    public double getBoomFilterFalsePositiveRate() {
+        return bloomFilterFalsePositiveRate;
+    }
+
     public IInMemoryFreePageManager getMemFreePageManager() {
         return memFreePageManager;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index 5860236..48cc481 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -72,7 +72,7 @@
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider, int ioDeviceId)
             throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
@@ -83,7 +83,7 @@
                 memFreePageManager, ioManager, file, diskBufferCache, diskFileMapProvider, typeTraits,
                 rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType, mergePolicy,
                 opTrackerFactory, ioScheduler, ioOpCallbackProvider,
-                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
+                LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length), ioDeviceId);
         LSMRTreeWithAntiMatterTuplesTestContext testCtx = new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes,
                 lsmTree);
         return testCtx;
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/NormalizedKeyComputer.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/NormalizedKeyComputer.java
new file mode 100644
index 0000000..6fd0fe5
--- /dev/null
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/NormalizedKeyComputer.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.api.graph;
+
+/**
+ * Users can extend this interface to speedup the performance, e.g., the Alpha-sort optimization for cache locality.
+ * The normalized key is an unsigned integer (represented by a signed integer, though) obtained from the binary represetnation
+ * of the corresponding vertex id.
+ * Usually the normalized key can be obtained from the prefix bytes of the vertex id bytes.
+ * 
+ * @author yingyib
+ */
+public interface NormalizedKeyComputer {
+
+    /**
+     * Get the normalized key from the byte region of a vertex id.
+     * The following three parameters represent the byte region of a vertex id.
+     * 
+     * @param data
+     * @param start
+     * @param len
+     * @return the normalized key.
+     */
+    public int getNormalizedKey(byte[] data, int start, int len);
+}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
index 8b6d1b6..dde1a5e 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
@@ -22,6 +22,7 @@
 
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
 import edu.uci.ics.pregelix.api.graph.MessageCombiner;
+import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
 import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.io.VertexInputFormat;
 import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
@@ -56,6 +57,8 @@
     public static final String PARTIAL_AGGREGATE_VALUE_CLASS = "pregelix.partialAggregateValueClass";
     /** Final aggregate value class */
     public static final String FINAL_AGGREGATE_VALUE_CLASS = "pregelix.finalAggregateValueClass";
+    /** The normalized key computer class */
+    public static final String NMK_COMPUTER_CLASS = "pregelix.nmkComputerClass";
     /** num of vertices */
     public static final String NUM_VERTICE = "pregelix.numVertices";
     /** num of edges */
@@ -166,4 +169,13 @@
     final public void setFrameSize(int frameSize) {
         getConfiguration().setInt(FRAME_SIZE, frameSize);
     }
+
+    /**
+     * Set the normalized key computer class
+     * 
+     * @param nkcClass
+     */
+    final public void setNoramlizedKeyComputerClass(Class<?> nkcClass) {
+        getConfiguration().setClass(NMK_COMPUTER_CLASS, nkcClass, NormalizedKeyComputer.class);
+    }
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
index ff9724d..6dfb416 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
 import edu.uci.ics.pregelix.api.graph.MessageCombiner;
 import edu.uci.ics.pregelix.api.graph.MsgList;
+import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
 import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.io.VertexInputFormat;
 import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
@@ -143,6 +144,18 @@
     }
 
     /**
+     * Create a user-defined normalized key computer class
+     * 
+     * @param conf
+     *            Configuration to check
+     * @return Instantiated user-defined normalized key computer
+     */
+    public static NormalizedKeyComputer createNormalizedKeyComputer(Configuration conf) {
+        Class<? extends NormalizedKeyComputer> nmkClass = getNormalizedKeyComputerClass(conf);
+        return ReflectionUtils.newInstance(nmkClass, conf);
+    }
+
+    /**
      * Create a global aggregator class
      * 
      * @param conf
@@ -320,7 +333,22 @@
     }
 
     /**
-     * Get the user's subclassed global aggregator's global value class.
+     * Get the user's subclassed normalized key computer class.
+     * 
+     * @param conf
+     *            Configuration to check
+     * @return User's normalized key computer class
+     */
+    @SuppressWarnings("unchecked")
+    public static Class<? extends NormalizedKeyComputer> getNormalizedKeyComputerClass(Configuration conf) {
+        if (conf == null)
+            conf = defaultConf;
+        return (Class<? extends NormalizedKeyComputer>) conf.getClass(PregelixJob.NMK_COMPUTER_CLASS,
+                NormalizedKeyComputer.class);
+    }
+
+    /**
+     * Get the user's subclassed normalized key computer class.
      * 
      * @param conf
      *            Configuration to check
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/INormalizedKeyComputerFactoryProvider.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/INormalizedKeyComputerFactoryProvider.java
deleted file mode 100644
index b6c995a..0000000
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/INormalizedKeyComputerFactoryProvider.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.pregelix.core.base;
-
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-
-public interface INormalizedKeyComputerFactoryProvider {
-
-    @SuppressWarnings("rawtypes")
-    INormalizedKeyComputerFactory getAscINormalizedKeyComputerFactory(Class keyClass);
-
-    @SuppressWarnings("rawtypes")
-    INormalizedKeyComputerFactory getDescINormalizedKeyComputerFactory(Class keyClass);
-}
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 ce1a34d..c913eff 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
@@ -70,7 +70,6 @@
 import edu.uci.ics.pregelix.core.data.TypeTraits;
 import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.jobgen.provider.NormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
@@ -173,6 +172,7 @@
                 new BTreeDataflowHelperFactory(), new TransientLocalResourceFactoryProvider(),
                 NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, btreeCreate);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -211,8 +211,7 @@
          */
         int[] sortFields = new int[1];
         sortFields[0] = 0;
-        INormalizedKeyComputerFactory nkmFactory = NormalizedKeyComputerFactoryProvider.INSTANCE
-                .getAscINormalizedKeyComputerFactory(vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
         comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -242,6 +241,7 @@
                 new WritableSerializerDeserializerFactory(vertexIdClass));
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sorter, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -287,8 +287,7 @@
          */
         int[] sortFields = new int[1];
         sortFields[0] = 0;
-        INormalizedKeyComputerFactory nkmFactory = NormalizedKeyComputerFactoryProvider.INSTANCE
-                .getAscINormalizedKeyComputerFactory(vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
         comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -319,6 +318,7 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
                 comparatorFactories), sorter, 0, writer, 0);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -444,6 +444,7 @@
          */
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
@@ -462,6 +463,7 @@
 
         ClusterConfig.setLocationConstraint(spec, drop);
         spec.addRoot(drop);
+        spec.setFrameSize(frameSize);
         return spec;
     }
 
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 ff2f6a0..9d23ca8 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
@@ -175,8 +175,7 @@
          * construct local sort operator
          */
         int[] keyFields = new int[] { 0 };
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -231,10 +230,11 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -385,8 +385,7 @@
         /**
          * construct local sort operator
          */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -455,10 +454,11 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index ee385f1..250d245 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -141,8 +141,7 @@
          * construct local sort operator
          */
         int[] keyFields = new int[] { 0 };
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -214,10 +213,11 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -345,8 +345,7 @@
         /**
          * construct local sort operator
          */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -416,9 +415,10 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
                 null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index 40b5f45..f796dd7 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -143,8 +143,7 @@
          * construct global sort operator
          */
         int[] keyFields = new int[] { 0 };
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -206,10 +205,11 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -334,8 +334,7 @@
         /**
          * construct global sort operator
          */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -393,10 +392,11 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index 3351a2c..cb6c215 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -140,8 +140,7 @@
          * construct local sort operator
          */
         int[] keyFields = new int[] { 0 };
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -220,10 +219,11 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
@@ -348,8 +348,7 @@
         /**
          * construct local sort operator
          */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil
-                .getINormalizedKeyComputerFactory(iteration, vertexIdClass);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
@@ -426,10 +425,11 @@
         /**
          * add the delete operator to delete vertexes
          */
+        int[] fieldPermutationDelete = new int[] { 0 };
         TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.DELETE, new BTreeDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
+                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
+                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, deleteOp);
 
         /** construct empty sink operator */
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
index 63b7c6d..319d7cc 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
@@ -15,10 +15,14 @@
 
 package edu.uci.ics.pregelix.core.jobgen;
 
+import org.apache.hadoop.conf.Configuration;
+
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.pregelix.core.jobgen.provider.NormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
+import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNormalizedKeyComputerFactory;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public class JobGenUtil {
@@ -31,8 +35,12 @@
      * @param keyClass
      * @return
      */
-    public static INormalizedKeyComputerFactory getINormalizedKeyComputerFactory(int iteration, Class keyClass) {
-        return NormalizedKeyComputerFactoryProvider.INSTANCE.getAscINormalizedKeyComputerFactory(keyClass);
+    public static INormalizedKeyComputerFactory getINormalizedKeyComputerFactory(Configuration conf) {
+        Class<? extends NormalizedKeyComputer> clazz = BspUtils.getNormalizedKeyComputerClass(conf);
+        if (clazz.equals(NormalizedKeyComputer.class)) {
+            return null;
+        }
+        return new VertexIdNormalizedKeyComputerFactory(clazz);
     }
 
     /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/provider/NormalizedKeyComputerFactoryProvider.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/provider/NormalizedKeyComputerFactoryProvider.java
deleted file mode 100644
index 0735593..0000000
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/provider/NormalizedKeyComputerFactoryProvider.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.pregelix.core.jobgen.provider;
-
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.pregelix.core.base.INormalizedKeyComputerFactoryProvider;
-import edu.uci.ics.pregelix.runtime.touchpoint.VLongAscNormalizedKeyComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VLongDescNormalizedKeyComputerFactory;
-
-public class NormalizedKeyComputerFactoryProvider implements INormalizedKeyComputerFactoryProvider {
-
-    public static INormalizedKeyComputerFactoryProvider INSTANCE = new NormalizedKeyComputerFactoryProvider();
-
-    private NormalizedKeyComputerFactoryProvider() {
-
-    }
-
-    @SuppressWarnings("rawtypes")
-    @Override
-    public INormalizedKeyComputerFactory getAscINormalizedKeyComputerFactory(Class keyClass) {
-        if (keyClass.getName().indexOf("VLongWritable") > 0)
-            return new VLongAscNormalizedKeyComputerFactory();
-        else
-            return null;
-    }
-
-    @SuppressWarnings("rawtypes")
-    @Override
-    public INormalizedKeyComputerFactory getDescINormalizedKeyComputerFactory(Class keyClass) {
-        if (keyClass.getName().indexOf("VLongWritable") > 0)
-            return new VLongDescNormalizedKeyComputerFactory();
-        else
-            return null;
-    }
-}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index d099645..8410e1e 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -52,7 +52,7 @@
         ccConfig.clusterNetPort = TEST_HYRACKS_CC_PORT;
         ccConfig.clientNetPort = TEST_HYRACKS_CC_CLIENT_PORT;
         ccConfig.defaultMaxJobAttempts = 0;
-        ccConfig.jobHistorySize = 0;
+        ccConfig.jobHistorySize = 1;
         ccConfig.profileDumpPeriod = -1;
 
         // cluster controller
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
index 3f89543..30df916 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
@@ -29,6 +29,17 @@
             UpdateBuffer updateBuffer, ArrayTupleBuilder cloneUpdateTb, ITreeIndexAccessor indexAccessor,
             ITreeIndexCursor cursor, RangePredicate rangePred) throws HyracksDataException, IndexException {
         if (cloneUpdateTb.getSize() > 0) {
+            int[] fieldEndOffsets = cloneUpdateTb.getFieldEndOffsets();
+            int srcStart = fieldEndOffsets[0];
+            int srcLen = fieldEndOffsets[1] - fieldEndOffsets[0]; // the updated vertex size
+            int frSize = frameTuple.getFieldLength(1); // the vertex binary size in the leaf page
+            if (srcLen <= frSize) {
+                //doing in-place update if possible, save the "real update" overhead
+                System.arraycopy(cloneUpdateTb.getByteArray(), srcStart, frameTuple.getFieldData(1),
+                        frameTuple.getFieldStart(1), srcLen);
+                cloneUpdateTb.reset();
+                return;
+            }
             if (!updateBuffer.appendTuple(cloneUpdateTb)) {
                 tempTupleReference.reset(frameTuple.getFieldData(0), frameTuple.getFieldStart(0),
                         frameTuple.getFieldLength(0));
@@ -49,5 +60,4 @@
             cloneUpdateTb.reset();
         }
     }
-
 }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
index 74ae455..6adbf83 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
@@ -34,6 +34,7 @@
 import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 
@@ -137,6 +138,7 @@
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         Client.run(args, job);
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
index e54373f..08de520 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 
@@ -40,6 +41,7 @@
 
     private VLongWritable vid = new VLongWritable();
     private GraphMutationVertex newVertex = null;
+    private DoubleWritable msg = new DoubleWritable(0.0);
 
     @Override
     public void compute(Iterator<DoubleWritable> msgIterator) {
@@ -47,17 +49,20 @@
             if (newVertex == null) {
                 newVertex = new GraphMutationVertex();
             }
-            if (getVertexId().get() % 2 == 0 || getVertexId().get() % 3 == 0) {
-                deleteVertex(getVertexId());
-            } else {
-                vid.set(100 * getVertexId().get());
-                newVertex.setVertexId(vid);
-                newVertex.setVertexValue(getVertexValue());
-                addVertex(vid, newVertex);
+            if (getVertexId().get() < 100) {
+                if ((getVertexId().get() % 2 == 0 || getVertexId().get() % 3 == 0)) {
+                    deleteVertex(getVertexId());
+                } else {
+                    vid.set(100 * getVertexId().get());
+                    newVertex.setVertexId(vid);
+                    newVertex.setVertexValue(getVertexValue());
+                    addVertex(vid, newVertex);
+                    sendMsg(vid, msg);
+                }
             }
             voteToHalt();
         } else {
-            if (getVertexId().get() % 190 == 0) {
+            if (getVertexId().get() == 1900) {
                 deleteVertex(getVertexId());
             }
             voteToHalt();
@@ -105,6 +110,7 @@
         job.setVertexClass(GraphMutationVertex.class);
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimpleGraphMutationVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         Client.run(args, job);
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
index b6d4da7..752b23a 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java
@@ -43,6 +43,7 @@
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 
@@ -216,6 +217,7 @@
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         Client.run(args, job);
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
index 0895386..8102c14 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ReachabilityVertex.java
@@ -40,6 +40,7 @@
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 
@@ -223,6 +224,7 @@
         job.setVertexInputFormatClass(TextReachibilityVertexInputFormat.class);
         job.setVertexOutputFormatClass(SimpleReachibilityVertexOutputFormat.class);
         job.setMessageCombinerClass(ReachabilityVertex.SimpleReachibilityCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         Client.run(args, job);
         System.out.println("reachable? " + readReachibilityResult(job.getConfiguration()));
     }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
index 199870e..396acae 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
 import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.inputformat.TextShortestPathsInputFormat;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 
@@ -138,6 +139,7 @@
         job.setVertexInputFormatClass(TextShortestPathsInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(ShortestPathsVertex.SimpleMinCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.getConfiguration().setLong(SOURCE_ID, 0);
         Client.run(args, job);
     }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
new file mode 100644
index 0000000..7d824ea
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.example.data;
+
+import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
+import edu.uci.ics.pregelix.api.util.SerDeUtils;
+
+/**
+ * @author yingyib
+ */
+public class VLongNormalizedKeyComputer implements NormalizedKeyComputer {
+
+    private static final int POSTIVE_LONG_MASK = (3 << 30);
+    private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
+    private static final int NEGATIVE_LONG_MASK = (0 << 30);
+
+    @Override
+    public int getNormalizedKey(byte[] bytes, int start, int length) {
+        long value = SerDeUtils.readVLong(bytes, start, length);
+        int highValue = (int) (value >> 32);
+        if (highValue > 0) {
+            /**
+             * larger than Integer.MAX
+             */
+            int highNmk = getKey(highValue);
+            highNmk >>= 2;
+            highNmk |= POSTIVE_LONG_MASK;
+            return highNmk;
+        } else if (highValue == 0) {
+            /**
+             * smaller than Integer.MAX but >=0
+             */
+            int lowNmk = (int) value;
+            lowNmk >>= 2;
+            lowNmk |= NON_NEGATIVE_INT_MASK;
+            return lowNmk;
+        } else {
+            /**
+             * less than 0; TODO: have not optimized for that
+             */
+            int highNmk = getKey(highValue);
+            highNmk >>= 2;
+            highNmk |= NEGATIVE_LONG_MASK;
+            return highNmk;
+        }
+    }
+
+    private int getKey(int value) {
+        long unsignedFirstValue = (long) value;
+        int nmk = (int) ((unsignedFirstValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+        return nmk;
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
index 85a139e..2ff800e 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java
@@ -39,6 +39,7 @@
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 import edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingVertex;
 
@@ -290,6 +291,7 @@
         job.setDynamicVertexValueSize(true);
         job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
         job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         Client.run(args, job);
         System.out.println("maximal cliques: \n" + readMaximalCliqueResult(job.getConfiguration()));
     }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java
index d3db095..89bfee8 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 
 /**
@@ -134,6 +135,7 @@
         job.setGlobalAggregatorClass(TriangleCountingAggregator.class);
         job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
         job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         Client.run(args, job);
         System.out.println("triangle count: " + readTriangleCountingResult(job.getConfiguration()));
     }
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
index 0a5b214..8fafede 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
@@ -33,6 +33,7 @@
 import edu.uci.ics.pregelix.example.ReachabilityVertex;
 import edu.uci.ics.pregelix.example.ReachabilityVertex.SimpleReachibilityVertexOutputFormat;
 import edu.uci.ics.pregelix.example.ShortestPathsVertex;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
 import edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat;
 import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
 import edu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat;
@@ -65,6 +66,7 @@
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -77,6 +79,7 @@
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -89,6 +92,7 @@
         job.setVertexInputFormatClass(TextShortestPathsInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(ShortestPathsVertex.SimpleMinCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -101,6 +105,7 @@
         job.setVertexClass(PageRankVertex.class);
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -113,6 +118,7 @@
         job.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class);
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -125,6 +131,7 @@
         job.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class);
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -137,6 +144,7 @@
         job.setVertexInputFormatClass(TextReachibilityVertexInputFormat.class);
         job.setVertexOutputFormatClass(SimpleReachibilityVertexOutputFormat.class);
         job.setMessageCombinerClass(ReachabilityVertex.SimpleReachibilityCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -152,6 +160,7 @@
         job.setVertexInputFormatClass(TextReachibilityVertexInputFormat.class);
         job.setVertexOutputFormatClass(SimpleReachibilityVertexOutputFormat.class);
         job.setMessageCombinerClass(ReachabilityVertex.SimpleReachibilityCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -166,6 +175,7 @@
         job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -178,6 +188,7 @@
         job.setVertexInputFormatClass(SimulatedPageRankVertexInputFormat.class);
         job.setMessageCombinerClass(ShortestPathsVertex.SimpleMinCombiner.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -191,6 +202,7 @@
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setDynamicVertexValueSize(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
@@ -204,6 +216,7 @@
         job.setGlobalAggregatorClass(TriangleCountingAggregator.class);
         job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class);
         job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
@@ -216,6 +229,7 @@
         job.setDynamicVertexValueSize(true);
         job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
         job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH3);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
@@ -228,6 +242,7 @@
         job.setDynamicVertexValueSize(true);
         job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
         job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH4);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
@@ -240,6 +255,7 @@
         job.setDynamicVertexValueSize(true);
         job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class);
         job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH5);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH3));
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
@@ -250,6 +266,7 @@
         job.setVertexClass(GraphMutationVertex.class);
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimpleGraphMutationVertexOutputFormat.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index 00f6f54..7126e8c 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -19,6 +19,7 @@
 
 import junit.framework.TestCase;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
@@ -93,17 +94,18 @@
     @Test
     public void test() throws Exception {
         setUp();
-        Plan[] plans = new Plan[] { Plan.OUTER_JOIN };
+        Plan[] plans = new Plan[] { Plan.OUTER_JOIN_SORT, Plan.OUTER_JOIN, Plan.INNER_JOIN, Plan.OUTER_JOIN_SINGLE_SORT };
         for (Plan plan : plans) {
             driver.runJob(job, plan, PregelixHyracksIntegrationUtil.CC_HOST,
                     PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
+            compareResults();
         }
-        compareResults();
         tearDown();
         waitawhile();
     }
 
     private void compareResults() throws Exception {
+        FileUtils.deleteQuietly(new File(resultFileDir));
         dfs.copyToLocalFile(FileOutputFormat.getOutputPath(job), new Path(resultFileDir));
         TestUtils.compareWithResultDir(new File(expectedFileDir), new File(resultFileDir));
     }
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
index 22ae6cf..decbde8 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>ConnectedComponents</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
index 50662f9..c7fec9f 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>ConnectedComponents</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
index 9f51f6d..d5ec8f1 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>Graph Mutation</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
index 616c647..c0559d9 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>Maximal Clique</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
index 5621259..541806d 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>Maximal Clique 2</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
index d4f81ba..7214b3f 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>Maximal Clique 3</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
index 744e5b0..65e0b30 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
index b51bd98..9e1e0b0 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
index a9e43bd..ceea85b 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
index c1a04ae..c05a4da 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
index 410ea8b..ac0d508 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
index 0332ec5..225429a 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>Reachibility</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
index 4f280fc..bd9da92 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>Reachibility</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
index 9e791e2..9acd7bc 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>ShortestPaths</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
index 90caf6b..6c25575 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>ShortestPaths</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
index 0f44f4d..4a40a6a 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml
@@ -37,6 +37,7 @@
 <property><name>fs.checkpoint.size</name><value>67108864</value></property>
 <property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
 <property><name>mapred.job.name</name><value>Triangle Counting</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
 <property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
 <property><name>mapred.userlog.limit.kb</name><value>0</value></property>
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
index 0a0a14f..caeeb10 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java
@@ -249,13 +249,10 @@
                     if (vertex != null && vertex.hasUpdate()) {
                         if (!dynamicStateLength) {
                             // in-place update
-                            int fieldCount = tupleRef.getFieldCount();
-                            for (int i = 1; i < fieldCount; i++) {
-                                byte[] data = tupleRef.getFieldData(i);
-                                int offset = tupleRef.getFieldStart(i);
-                                bbos.setByteArray(data, offset);
-                                vertex.write(output);
-                            }
+                            byte[] data = tupleRef.getFieldData(1);
+                            int offset = tupleRef.getFieldStart(1);
+                            bbos.setByteArray(data, offset);
+                            vertex.write(output);
                         } else {
                             // write the vertex id
                             DataOutput tbOutput = cloneUpdateTb.getDataOutput();
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
index 9998205..48d4d80 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java
@@ -253,13 +253,10 @@
                     if (vertex != null && vertex.hasUpdate()) {
                         if (!dynamicStateLength) {
                             // in-place update
-                            int fieldCount = tupleRef.getFieldCount();
-                            for (int i = 1; i < fieldCount; i++) {
-                                byte[] data = tupleRef.getFieldData(i);
-                                int offset = tupleRef.getFieldStart(i);
-                                bbos.setByteArray(data, offset);
-                                vertex.write(output);
-                            }
+                            byte[] data = tupleRef.getFieldData(1);
+                            int offset = tupleRef.getFieldStart(1);
+                            bbos.setByteArray(data, offset);
+                            vertex.write(output);
                         } else {
                             // write the vertex id
                             DataOutput tbOutput = cloneUpdateTb.getDataOutput();
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VLongAscNormalizedKeyComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VLongAscNormalizedKeyComputerFactory.java
deleted file mode 100644
index 9181691..0000000
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VLongAscNormalizedKeyComputerFactory.java
+++ /dev/null
@@ -1,56 +0,0 @@
-package edu.uci.ics.pregelix.runtime.touchpoint;
-
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.pregelix.api.util.SerDeUtils;
-
-public class VLongAscNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
-        return new INormalizedKeyComputer() {
-            private static final int POSTIVE_LONG_MASK = (3 << 30);
-            private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
-            private static final int NEGATIVE_LONG_MASK = (0 << 30);
-
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                long value = SerDeUtils.readVLong(bytes, start, length);
-                int highValue = (int) (value >> 32);
-                if (highValue > 0) {
-                    /**
-                     * larger than Integer.MAX
-                     */
-                    int highNmk = getKey(highValue);
-                    highNmk >>= 2;
-                    highNmk |= POSTIVE_LONG_MASK;
-                    return highNmk;
-                } else if (highValue == 0) {
-                    /**
-                     * smaller than Integer.MAX but >=0
-                     */
-                    int lowNmk = (int) value;
-                    lowNmk >>= 2;
-                    lowNmk |= NON_NEGATIVE_INT_MASK;
-                    return lowNmk;
-                } else {
-                    /**
-                     * less than 0; TODO: have not optimized for that
-                     */
-                    int highNmk = getKey(highValue);
-                    highNmk >>= 2;
-                    highNmk |= NEGATIVE_LONG_MASK;
-                    return highNmk;
-                }
-            }
-
-            private int getKey(int value) {
-                long unsignedFirstValue = (long) value;
-                int nmk = (int) ((unsignedFirstValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
-                return nmk;
-            }
-
-        };
-    }
-}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VLongDescNormalizedKeyComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VLongDescNormalizedKeyComputerFactory.java
deleted file mode 100644
index 6b2738b..0000000
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VLongDescNormalizedKeyComputerFactory.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package edu.uci.ics.pregelix.runtime.touchpoint;
-
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-
-public class VLongDescNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
-    private static final long serialVersionUID = 1L;
-    private final INormalizedKeyComputerFactory ascNormalizedKeyComputerFactory = new VLongAscNormalizedKeyComputerFactory();
-
-    @Override
-    public INormalizedKeyComputer createNormalizedKeyComputer() {
-        return new INormalizedKeyComputer() {
-            private INormalizedKeyComputer nmkComputer = ascNormalizedKeyComputerFactory.createNormalizedKeyComputer();
-
-            @Override
-            public int normalize(byte[] bytes, int start, int length) {
-                int nk = nmkComputer.normalize(bytes, start, length);
-                return (int) ((long) Integer.MAX_VALUE - (long) (nk - Integer.MIN_VALUE));
-            }
-
-        };
-    }
-}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdNormalizedKeyComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..04e16ac
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdNormalizedKeyComputerFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.runtime.touchpoint;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
+
+/**
+ * This class wraps the user-defined normalized key computer to calculate the normalized
+ * key of vertex ids.
+ * 
+ * @author yingyib
+ */
+public class VertexIdNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private Class<? extends NormalizedKeyComputer> nmkComputerClass;
+
+    public VertexIdNormalizedKeyComputerFactory(Class<? extends NormalizedKeyComputer> nmkComputerClass) {
+        this.nmkComputerClass = nmkComputerClass;
+    }
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        try {
+            final NormalizedKeyComputer nmkComputer = nmkComputerClass.newInstance();
+            return new INormalizedKeyComputer() {
+
+                @Override
+                public int normalize(byte[] bytes, int start, int length) {
+                    return nmkComputer.getNormalizedKey(bytes, start, length);
+                }
+
+            };
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+}