Merge branch 'master' into raman/master_feeds
diff --git a/.gitignore b/.gitignore
index 0517ac2..36befc4 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,3 +1,4 @@
+bin
 target
 .classpath
 .settings
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestOperator.java
index 7efee75..b9ac62f 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestOperator.java
@@ -23,12 +23,22 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+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.visitors.ILogicalOperatorVisitor;
 
 public class UnnestOperator extends AbstractUnnestOperator {
 
     private LogicalVariable positionalVariable;
+
+    /**
+     * Used to set the position offset for positional variable
+     */
+    private ILogicalExpression positionOffsetExpr;
+
+    /**
+     * Specify the type of the positional variable
+     */
     private Object positionalVariableType;
 
     public UnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression) {
@@ -67,6 +77,14 @@
         return positionalVariableType;
     }
 
+    public void setPositionOffsetExpr(ILogicalExpression posOffsetExpr) {
+        this.positionOffsetExpr = posOffsetExpr;
+    }
+
+    public ILogicalExpression getPositionOffsetExpr() {
+        return this.positionOffsetExpr;
+    }
+
     @Override
     public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
         return visitor.visitUnnestOperator(this, arg);
@@ -88,4 +106,24 @@
         }
         return env;
     }
+
+    @Override
+    public VariablePropagationPolicy getVariablePropagationPolicy() {
+        return new VariablePropagationPolicy() {
+
+            @Override
+            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+                    throws AlgebricksException {
+                if (sources.length > 0) {
+                    target.addAllVariables(sources[0]);
+                }
+                for (LogicalVariable v : variables) {
+                    target.addVariable(v);
+                }
+                if (positionalVariable != null) {
+                    target.addVariable(positionalVariable);
+                }
+            }
+        };
+    }
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index cebddee..3a2617f 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -404,7 +404,8 @@
     @Override
     public Void visitRunningAggregateOperator(RunningAggregateOperator op, IOptimizationContext ctx)
             throws AlgebricksException {
-        propagateFDsAndEquivClasses(op, ctx);
+        ctx.putEquivalenceClassMap(op, new HashMap<LogicalVariable, EquivalenceClass>());
+        ctx.putFDList(op, new ArrayList<FunctionalDependency>());
         return null;
     }
 
@@ -646,4 +647,4 @@
         return null;
     }
 
-}
+}
\ No newline at end of file
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
index 9955a63..3f1aae2 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
@@ -44,9 +44,11 @@
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 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.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -142,27 +144,32 @@
         int n = orderColumns.size();
         int[] sortFields = new int[n];
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[n];
-        {
-            int j = 0;
-            for (OrderColumn oc : orderColumns) {
-                LogicalVariable var = oc.getColumn();
-                sortFields[j] = opSchema.findVariable(var);
-                Object type = env.getVarType(var);
-                IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
-                comparatorFactories[j] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
-                j++;
+
+        INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+        INormalizedKeyComputerFactory nkcf = null;
+
+        int j = 0;
+        for (OrderColumn oc : orderColumns) {
+            LogicalVariable var = oc.getColumn();
+            sortFields[j] = opSchema.findVariable(var);
+            Object type = env.getVarType(var);
+            IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+            comparatorFactories[j] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
+            if (j == 0 && nkcfProvider != null && type != null) {
+                nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, oc.getOrder() == OrderKind.ASC);
             }
+            j++;
         }
 
         IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields,
-                comparatorFactories);
+                comparatorFactories, nkcf);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
     }
-    
+
     public List<LogicalVariable> getPartitionFields() {
         return partitionFields;
     }
-    
+
     public List<OrderColumn> getOrderColumns() {
         return orderColumns;
     }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
index 51b009f..bcd998d 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
@@ -34,6 +35,7 @@
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import edu.uci.ics.hyracks.algebricks.runtime.base.AlgebricksPipeline;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.aggreg.NestedPlansAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.aggreg.NestedPlansRunningAggregatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -76,13 +78,19 @@
         }
         // compile subplans and set the gby op. schema accordingly
         AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], gby, opSchema, context);
-        IAggregatorDescriptorFactory aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys,
-                fdColumns);
+        IAggregatorDescriptorFactory aggregatorFactory;
+
+        if (((AbstractLogicalOperator) (gby.getNestedPlans().get(0).getRoots().get(0).getValue())).getOperatorTag() == LogicalOperatorTag.RUNNINGAGGREGATE) {
+            aggregatorFactory = new NestedPlansRunningAggregatorFactory(subplans, keys, fdColumns);
+        } else {
+            aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys, fdColumns);
+        }
 
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
                 columnList, context.getTypeEnvironment(op), context);
-        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
+        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema,
+                context);
 
         PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
                 comparatorFactories, aggregatorFactory, recordDescriptor);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
index 21def02..956c74f 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
@@ -62,22 +62,7 @@
     @Override
     public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
             IPhysicalPropertiesVector reqdByParent) {
-        IPartitioningProperty pp = null;
-        RunningAggregateOperator ragg = (RunningAggregateOperator) op;
-        for (Mutable<ILogicalExpression> exprRef : ragg.getExpressions()) {
-            StatefulFunctionCallExpression f = (StatefulFunctionCallExpression) exprRef.getValue();
-            IPartitioningProperty p = f.getRequiredPartitioningProperty();
-            if (p != null) {
-                if (pp == null) {
-                    pp = p;
-                } else {
-                    throw new IllegalStateException("Two stateful functions want to set partitioning requirements: "
-                            + pp + " and " + p);
-                }
-            }
-        }
-        StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp, null) };
-        return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+        return emptyUnaryRequirements();
     }
 
     @Override
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
index dc345ac..b03b99d 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
@@ -39,9 +39,11 @@
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 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.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -128,6 +130,10 @@
         IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
         IBinaryHashFunctionFactory[] hashFuns = new IBinaryHashFunctionFactory[n];
         IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+
+        INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+        INormalizedKeyComputerFactory nkcf = null;
+
         for (int i = 0; i < n; i++) {
             sortFields[i] = opSchema.findVariable(sortColumns[i].getColumn());
             Object type = env.getVarType(sortColumns[i].getColumn());
@@ -135,9 +141,12 @@
             comps[i] = bcfp.getBinaryComparatorFactory(type, sortColumns[i].getOrder() == OrderKind.ASC);
             IBinaryHashFunctionFactoryProvider bhffp = context.getBinaryHashFunctionFactoryProvider();
             hashFuns[i] = bhffp.getBinaryHashFunctionFactory(type);
+            if (i == 0 && nkcfProvider != null && type != null) {
+                nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, sortColumns[i].getOrder() == OrderKind.ASC);
+            }
         }
         ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(sortFields, hashFuns);
-        IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps);
+        IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, TargetConstraint.ONE);
     }
 
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
index bf45e18..bf2f68a 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 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.IScalarEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -57,10 +58,7 @@
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
             throws AlgebricksException {
         UnnestOperator unnest = (UnnestOperator) op;
-        if (unnest.getPositionalVariable() != null) {
-            throw new AlgebricksException("Cannot generate runtime for unnest with positional variable "
-                    + unnest.getPositionalVariable());
-        }
+
         int outCol = opSchema.findVariable(unnest.getVariable());
         ILogicalExpression unnestExpr = unnest.getExpressionRef().getValue();
         IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
@@ -79,8 +77,18 @@
         UnnestingFunctionCallExpression agg = (UnnestingFunctionCallExpression) unnestExpr;
         IUnnestingEvaluatorFactory unnestingFactory = expressionRuntimeProvider.createUnnestingFunctionFactory(agg,
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas, context);
+
+        // for position offset
+        ILogicalExpression posOffsetExpr = unnest.getPositionOffsetExpr();
+        IScalarEvaluatorFactory posOffsetExprEvalFactory = null;
+        if (posOffsetExpr != null) {
+            posOffsetExprEvalFactory = expressionRuntimeProvider.createEvaluatorFactory(posOffsetExpr,
+                    context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas, context);
+        }
+
         int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
-        UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList);
+        UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList,
+                unnest.getPositionalVariable() != null, posOffsetExprEvalFactory);
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
         ILogicalOperator src = unnest.getInputs().get(0).getValue();
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 a684bc9..02e090d 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
@@ -104,6 +104,10 @@
                     foundTarget = false;
                     break;
                 }
+                if(child.getOperatorTag() == LogicalOperatorTag.GROUP){
+                    foundTarget = false;
+                    break;
+                }
                 if (orderSensitiveOps.contains(child.getOperatorTag())) {
                     orderSensitive = true;
                 }
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 98606f1..810defc 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -270,6 +270,8 @@
             // Now, transfer annotations from the original sort op. to this one.
             AbstractLogicalOperator transferTo = nextOp;
             if (transferTo.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+                // 
+                // remove duplicate exchange operator
                 transferTo = (AbstractLogicalOperator) transferTo.getInputs().get(0).getValue();
             }
             transferTo.getAnnotations().putAll(op.getAnnotations());
@@ -345,6 +347,13 @@
             LocalOrderProperty orderProp = (LocalOrderProperty) dlvd.get(j);
             returnedProperties.add(new OrderColumn(orderProp.getColumn(), orderProp.getOrder()));
         }
+        // maintain other order columns after the required order columns
+        if(returnedProperties.size() != 0){
+            for(int j = prefix + 1; j < dlvdCols.size(); j++){
+                LocalOrderProperty orderProp = (LocalOrderProperty) dlvd.get(j);
+                returnedProperties.add(new OrderColumn(orderProp.getColumn(), orderProp.getOrder()));
+            }
+        }
         return returnedProperties;
     }
 
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java
index af11b70..e702d9f 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/IntroHashPartitionMergeExchange.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 public class IntroHashPartitionMergeExchange implements IAlgebraicRewriteRule {
@@ -42,7 +43,8 @@
             throws AlgebricksException {
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
         if (op1.getPhysicalOperator() == null
-                || op1.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_EXCHANGE) {
+                || (op1.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_EXCHANGE && op1
+                        .getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_MERGE_EXCHANGE)) {
             return false;
         }
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
@@ -50,6 +52,12 @@
                 || op2.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.SORT_MERGE_EXCHANGE) {
             return false;
         }
+        if (op1.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.HASH_PARTITION_MERGE_EXCHANGE) {
+            // if it is a hash_partition_merge_exchange, the sort_merge_exchange can be simply removed
+            op1.getInputs().get(0).setValue(op2.getInputs().get(0).getValue());
+            op1.computeDeliveredPhysicalProperties(context);
+            return true;
+        }
         HashPartitionExchangePOperator hpe = (HashPartitionExchangePOperator) op1.getPhysicalOperator();
         SortMergeExchangePOperator sme = (SortMergeExchangePOperator) op2.getPhysicalOperator();
         List<OrderColumn> ocList = new ArrayList<OrderColumn>();
@@ -60,6 +68,7 @@
                 hpe.getDomain());
         op1.setPhysicalOperator(hpme);
         op1.getInputs().get(0).setValue(op2.getInputs().get(0).getValue());
+        op1.computeDeliveredPhysicalProperties(context);
         return true;
     }
 
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
index 2a0f6c3..756e0f3 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -48,8 +48,7 @@
             throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
 
-        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN
-                && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) {
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
             return false;
         }
         AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index 8f2eaac..09354a5 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -28,11 +28,11 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.group.AbstractAccumulatingAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
-public class NestedPlansAccumulatingAggregatorFactory implements IAggregatorDescriptorFactory {
+public class NestedPlansAccumulatingAggregatorFactory extends AbstractAccumulatingAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
     private AlgebricksPipeline[] subplans;
@@ -95,7 +95,7 @@
             }
 
             @Override
-            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 for (int i = 0; i < pipelines.length; i++) {
                     outputWriter.setInputIdx(i);
@@ -114,6 +114,7 @@
                     offset = fieldEnds[i] - start;
                     tupleBuilder.addField(data, start, offset);
                 }
+                return true;
             }
 
             @Override
@@ -127,7 +128,7 @@
             }
 
             @Override
-            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 throw new IllegalStateException("this method should not be called");
             }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
new file mode 100644
index 0000000..8973c75
--- /dev/null
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
@@ -0,0 +1,262 @@
+/*
+ * 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.algebricks.runtime.operators.aggreg;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+
+public class NestedPlansRunningAggregatorFactory implements IAggregatorDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+    private AlgebricksPipeline[] subplans;
+    private int[] keyFieldIdx;
+    private int[] decorFieldIdx;
+
+    public NestedPlansRunningAggregatorFactory(AlgebricksPipeline[] subplans, int[] keyFieldIdx, int[] decorFieldIdx) {
+        this.subplans = subplans;
+        this.keyFieldIdx = keyFieldIdx;
+        this.decorFieldIdx = decorFieldIdx;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory#createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext, edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int[], int[])
+     */
+    @Override
+    public IAggregatorDescriptor createAggregator(final IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, int[] keyFields, int[] keyFieldsInPartialResults,
+            final IFrameWriter writer) throws HyracksDataException {
+        final RunningAggregatorOutput outputWriter = new RunningAggregatorOutput(ctx, subplans, keyFieldIdx.length,
+                decorFieldIdx.length, writer);
+        final NestedTupleSourceRuntime[] pipelines = new NestedTupleSourceRuntime[subplans.length];
+        for (int i = 0; i < subplans.length; i++) {
+            try {
+                pipelines[i] = (NestedTupleSourceRuntime) assemblePipeline(subplans[i], outputWriter, ctx);
+            } catch (AlgebricksException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        final ArrayTupleBuilder gbyTb = outputWriter.getGroupByTupleBuilder();
+
+        final ByteBuffer outputFrame = ctx.allocateFrame();
+        final FrameTupleAppender outputAppender = new FrameTupleAppender(ctx.getFrameSize());
+        outputAppender.reset(outputFrame, true);
+
+        return new IAggregatorDescriptor() {
+
+            @Override
+            public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+
+                for (int i = 0; i < pipelines.length; ++i) {
+                    pipelines[i].open();
+                }
+
+                gbyTb.reset();
+                for (int i = 0; i < keyFieldIdx.length; ++i) {
+                    gbyTb.addField(accessor, tIndex, keyFieldIdx[i]);
+                }
+                for (int i = 0; i < decorFieldIdx.length; ++i) {
+                    gbyTb.addField(accessor, tIndex, decorFieldIdx[i]);
+                }
+
+                // aggregate the first tuple
+                for (int i = 0; i < pipelines.length; i++) {
+                    outputWriter.setInputIdx(i);
+                    pipelines[i].writeTuple(accessor.getBuffer(), tIndex);
+                    pipelines[i].forceFlush();
+                }
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
+                    int stateTupleIndex, AggregateState state) throws HyracksDataException {
+                for (int i = 0; i < pipelines.length; i++) {
+                    outputWriter.setInputIdx(i);
+                    pipelines[i].writeTuple(accessor.getBuffer(), tIndex);
+                    pipelines[i].forceFlush();
+                }
+            }
+
+            @Override
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                return false;
+            }
+
+            @Override
+            public AggregateState createAggregateStates() {
+                return new AggregateState();
+            }
+
+            @Override
+            public void reset() {
+
+            }
+
+            @Override
+            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor,
+                    int tIndex, AggregateState state) throws HyracksDataException {
+                throw new IllegalStateException("this method should not be called");
+            }
+
+            @Override
+            public void close() {
+                for (int i = 0; i < pipelines.length; ++i) {
+                    try {
+                        outputWriter.setInputIdx(i);
+                        pipelines[i].close();
+                    } catch (HyracksDataException e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            }
+        };
+    }
+
+    private IFrameWriter assemblePipeline(AlgebricksPipeline subplan, IFrameWriter writer, IHyracksTaskContext ctx)
+            throws AlgebricksException, HyracksDataException {
+        // plug the operators
+        IFrameWriter start = writer;
+        IPushRuntimeFactory[] runtimeFactories = subplan.getRuntimeFactories();
+        RecordDescriptor[] recordDescriptors = subplan.getRecordDescriptors();
+        for (int i = runtimeFactories.length - 1; i >= 0; i--) {
+            IPushRuntime newRuntime = runtimeFactories[i].createPushRuntime(ctx);
+            newRuntime.setFrameWriter(0, start, recordDescriptors[i]);
+            if (i > 0) {
+                newRuntime.setInputRecordDescriptor(0, recordDescriptors[i - 1]);
+            } else {
+                // the nts has the same input and output rec. desc.
+                newRuntime.setInputRecordDescriptor(0, recordDescriptors[0]);
+            }
+            start = newRuntime;
+        }
+        return start;
+    }
+
+    private static class RunningAggregatorOutput implements IFrameWriter {
+
+        private FrameTupleAccessor[] tAccess;
+        private RecordDescriptor[] inputRecDesc;
+        private int inputIdx;
+        private ArrayTupleBuilder tb;
+        private ArrayTupleBuilder gbyTb;
+        private AlgebricksPipeline[] subplans;
+        private IFrameWriter outputWriter;
+        private ByteBuffer outputFrame;
+        private FrameTupleAppender outputAppender;
+
+        public RunningAggregatorOutput(IHyracksTaskContext ctx, AlgebricksPipeline[] subplans, int numKeys,
+                int numDecors, IFrameWriter outputWriter) throws HyracksDataException {
+            this.subplans = subplans;
+            this.outputWriter = outputWriter;
+
+            // this.keyFieldIndexes = keyFieldIndexes;
+            int totalAggFields = 0;
+            this.inputRecDesc = new RecordDescriptor[subplans.length];
+            for (int i = 0; i < subplans.length; i++) {
+                RecordDescriptor[] rd = subplans[i].getRecordDescriptors();
+                this.inputRecDesc[i] = rd[rd.length - 1];
+                totalAggFields += subplans[i].getOutputWidth();
+            }
+            tb = new ArrayTupleBuilder(numKeys + numDecors + totalAggFields);
+            gbyTb = new ArrayTupleBuilder(numKeys + numDecors);
+
+            this.tAccess = new FrameTupleAccessor[inputRecDesc.length];
+            for (int i = 0; i < inputRecDesc.length; i++) {
+                tAccess[i] = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc[i]);
+            }
+
+            this.outputFrame = ctx.allocateFrame();
+            this.outputAppender = new FrameTupleAppender(ctx.getFrameSize());
+            this.outputAppender.reset(outputFrame, true);
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            int w = subplans[inputIdx].getOutputWidth();
+            IFrameTupleAccessor accessor = tAccess[inputIdx];
+            accessor.reset(buffer);
+            for (int tIndex = 0; tIndex < accessor.getTupleCount(); tIndex++) {
+                tb.reset();
+                byte[] data = gbyTb.getByteArray();
+                int[] fieldEnds = gbyTb.getFieldEndOffsets();
+                int start = 0;
+                int offset = 0;
+                for (int i = 0; i < fieldEnds.length; i++) {
+                    if (i > 0)
+                        start = fieldEnds[i - 1];
+                    offset = fieldEnds[i] - start;
+                    tb.addField(data, start, offset);
+                }
+                for (int f = 0; f < w; f++) {
+                    tb.addField(accessor, tIndex, f);
+                }
+                if (!outputAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    FrameUtils.flushFrame(outputFrame, outputWriter);
+                    outputAppender.reset(outputFrame, true);
+                    if (!outputAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                        throw new HyracksDataException(
+                                "Failed to write a running aggregation result into an empty frame: possibly the size of the result is too large.");
+                    }
+                }
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            if(outputAppender.getTupleCount() > 0){
+                FrameUtils.flushFrame(outputFrame, outputWriter);
+                outputAppender.reset(outputFrame, true);
+            }
+        }
+
+        public void setInputIdx(int inputIdx) {
+            this.inputIdx = inputIdx;
+        }
+
+        public ArrayTupleBuilder getGroupByTupleBuilder() {
+            return gbyTb;
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+        }
+
+    }
+
+}
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
index d4161fc..39db3e1 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
@@ -25,11 +25,11 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.group.AbstractAccumulatingAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
-public class SerializableAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+public class SerializableAggregatorDescriptorFactory extends AbstractAccumulatingAggregatorDescriptorFactory {
     private static final long serialVersionUID = 1L;
     private ICopySerializableAggregateFunctionFactory[] aggFactories;
 
@@ -109,7 +109,7 @@
             }
 
             @Override
-            public void outputPartialResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputPartialResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 byte[] data = accessor.getBuffer().array();
                 int startOffset = accessor.getTupleStartOffset(tIndex);
@@ -125,10 +125,11 @@
                         throw new HyracksDataException(e);
                     }
                 }
+                return true;
             }
 
             @Override
-            public void outputFinalResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 byte[] data = accessor.getBuffer().array();
                 int startOffset = accessor.getTupleStartOffset(tIndex);
@@ -144,6 +145,7 @@
                         throw new HyracksDataException(e);
                     }
                 }
+                return true;
             }
 
             @Override
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 8e472ec..9b638b4 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
@@ -25,11 +25,11 @@
 import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.group.AbstractAccumulatingAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
-public class SimpleAlgebricksAccumulatingAggregatorFactory implements IAggregatorDescriptorFactory {
+public class SimpleAlgebricksAccumulatingAggregatorFactory extends AbstractAccumulatingAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
     private IAggregateEvaluatorFactory[] aggFactories;
@@ -86,7 +86,7 @@
             }
 
             @Override
-            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 IAggregateEvaluator[] agg = (IAggregateEvaluator[]) state.state;
                 for (int i = 0; i < agg.length; i++) {
@@ -97,6 +97,7 @@
                         throw new HyracksDataException(e);
                     }
                 }
+                return true;
             }
 
             @Override
@@ -118,7 +119,7 @@
             }
 
             @Override
-            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 throw new IllegalStateException("this method should not be called");
             }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
index 6c1dd5e..b079c3e 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -51,8 +51,8 @@
             FrameUtils.flushFrame(frame, writer);
             appender.reset(frame, true);
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new IllegalStateException(
-                        "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime.appendToFrameFromTupleBuilder).");
+                throw new HyracksDataException(
+                        "Could not write frame: the size of the tuple is too long to be fit into a single frame. (AbstractOneInputOneOutputOneFramePushRuntime.appendToFrameFromTupleBuilder)");
             }
         }
         if (flushFrame) {
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
index 200e5c5..7ecb288 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
@@ -63,8 +63,6 @@
             for (int i = 0; i < comparatorFactories.length; ++i) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
-            final IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
-                    outRecordDesc, groupFields, groupFields);
             final ByteBuffer copyFrame = ctx.allocateFrame();
             final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
             copyFrameAccessor.reset(copyFrame);
@@ -78,6 +76,8 @@
 
                 @Override
                 public void open() throws HyracksDataException {
+                    IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
+                            outRecordDesc, groupFields, groupFields, writer);
                     pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregator, inRecordDesc,
                             outRecordDesc, writer);
                     pgw.open();
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 0499684..e5bede2 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -107,7 +107,8 @@
                             FrameUtils.flushFrame(frame, writer);
                             appender.reset(frame, true);
                             if (!appender.appendConcat(tRef.getFrameTupleAccessor(), tRef.getTupleIndex(), ta, t)) {
-                                throw new IllegalStateException("Could not write frame.");
+                                throw new HyracksDataException(
+                                        "Could not write frame: subplan result is larger than the single-frame limit.");
                             }
                         }
                     }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
index b299e78..131eea0 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -24,7 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterMergeSort;
 
 public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
@@ -52,12 +52,12 @@
 
         return new AbstractOneInputOneOutputPushRuntime() {
 
-            FrameSorter frameSorter = null;
+            FrameSorterMergeSort frameSorter = null;
 
             @Override
             public void open() throws HyracksDataException {
                 if (frameSorter == null) {
-                    frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
                             outputRecordDesc);
                 }
                 frameSorter.reset();
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
index 5486fae..55dbcbb 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class NestedTupleSourceRuntimeFactory implements IPushRuntimeFactory {
 
@@ -64,5 +65,10 @@
         public void fail() throws HyracksDataException {
             writer.fail();
         }
+
+        public void forceFlush() throws HyracksDataException {
+            FrameUtils.flushFrame(frame, writer);
+            appender.reset(frame, true);
+        }
     }
 }
\ No newline at end of file
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
index e92724a..600d641 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 
 public class RunningAggregateRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
@@ -89,6 +90,10 @@
 
             @Override
             public void open() throws HyracksDataException {
+                if (!first) {
+                    FrameUtils.flushFrame(frame, writer);
+                    appender.reset(frame, true);
+                }
                 initAccessAppendRef(ctx);
                 if (first) {
                     first = false;
@@ -96,12 +101,18 @@
                     for (int i = 0; i < n; i++) {
                         try {
                             raggs[i] = runningAggregates[i].createRunningAggregateEvaluator();
-                            raggs[i].init();
                         } catch (AlgebricksException ae) {
                             throw new HyracksDataException(ae);
                         }
                     }
                 }
+                for (int i = 0; i < runningAggregates.length; i++) {
+                    try {
+                        raggs[i].init();
+                    } catch (AlgebricksException ae) {
+                        throw new HyracksDataException(ae);
+                    }
+                }
                 writer.open();
             }
 
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
index df70437..415e718 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
@@ -14,11 +14,15 @@
  */
 package edu.uci.ics.hyracks.algebricks.runtime.operators.std;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.evaluators.ConstantEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -26,6 +30,7 @@
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 
 public class UnnestRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
@@ -36,12 +41,20 @@
     private int outColPos;
     private final boolean outColIsProjected;
 
+    private final boolean hasPositionalVariable;
+    private IScalarEvaluatorFactory posOffsetEvalFactory;
+
     // Each time step() is called on the aggregate, a new value is written in
     // its output. One byte is written before that value and is neglected.
     // By convention, if the aggregate function writes nothing, it means it
     // produced the last value.
 
     public UnnestRuntimeFactory(int outCol, IUnnestingEvaluatorFactory unnestingFactory, int[] projectionList) {
+        this(outCol, unnestingFactory, projectionList, false, null);
+    }
+
+    public UnnestRuntimeFactory(int outCol, IUnnestingEvaluatorFactory unnestingFactory, int[] projectionList,
+            boolean hashPositionalVariable, IScalarEvaluatorFactory posOffsetEvalFactory) {
         super(projectionList);
         this.outCol = outCol;
         this.unnestingFactory = unnestingFactory;
@@ -52,6 +65,11 @@
             }
         }
         outColIsProjected = outColPos >= 0;
+        this.hasPositionalVariable = hashPositionalVariable;
+        this.posOffsetEvalFactory = posOffsetEvalFactory;
+        if (this.posOffsetEvalFactory == null) {
+            this.posOffsetEvalFactory = new ConstantEvaluatorFactory(new byte[5]);
+        }
     }
 
     @Override
@@ -68,6 +86,8 @@
             private IUnnestingEvaluator agg;
             private ArrayTupleBuilder tupleBuilder;
 
+            private IScalarEvaluator offsetEval = posOffsetEvalFactory.createScalarEvaluator(ctx);
+
             @Override
             public void open() throws HyracksDataException {
                 initAccessAppendRef(ctx);
@@ -86,29 +106,55 @@
                 int nTuple = tAccess.getTupleCount();
                 for (int t = 0; t < nTuple; t++) {
                     tRef.reset(tAccess, t);
+
+                    try {
+                        offsetEval.evaluate(tRef, p);
+                    } catch (AlgebricksException e) {
+                        throw new HyracksDataException(e);
+                    }
+
+                    @SuppressWarnings("static-access")
+                    int offset = IntegerSerializerDeserializer.INSTANCE.getInt(p.getByteArray(), p.getStartOffset());
+
                     try {
                         agg.init(tRef);
+                        // assume that when unnesting the tuple, each step() call for each element
+                        // in the tuple will increase the positionIndex, and the positionIndex will
+                        // be reset when a new tuple is to be processed.
+                        int positionIndex = 1;
                         boolean goon = true;
                         do {
                             tupleBuilder.reset();
                             if (!agg.step(p)) {
                                 goon = false;
                             } else {
-                                if (!outColIsProjected) {
+
+                                if (!outColIsProjected && !hasPositionalVariable) {
                                     appendProjectionToFrame(t, projectionList);
                                 } else {
                                     for (int f = 0; f < outColPos; f++) {
                                         tupleBuilder.addField(tAccess, t, f);
                                     }
-                                    tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
-                                    for (int f = outColPos + 1; f < projectionList.length; f++) {
+                                    if (outColIsProjected) {
+                                        tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
+                                    } else {
+                                        tupleBuilder.addField(tAccess, t, outColPos);
+                                    }
+                                    for (int f = outColPos + 1; f < (hasPositionalVariable ? projectionList.length - 1
+                                            : projectionList.length); f++) {
                                         tupleBuilder.addField(tAccess, t, f);
                                     }
                                 }
+                                if (hasPositionalVariable) {
+                                    // Write the positional variable as an INT32
+                                    tupleBuilder.getDataOutput().writeByte(3);
+                                    tupleBuilder.getDataOutput().writeInt(offset + positionIndex++);
+                                    tupleBuilder.addFieldEndOffset();
+                                }
                                 appendToFrameFromTupleBuilder(tupleBuilder);
                             }
                         } while (goon);
-                    } catch (AlgebricksException ae) {
+                    } catch (AlgebricksException | IOException ae) {
                         throw new HyracksDataException(ae);
                     }
                 }
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan
index a22bf53..e4f2cd6 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan
@@ -14,7 +14,7 @@
                }
         -- PRE_CLUSTERED_GROUP_BY[$$17]  |PARTITIONED|
           exchange 
-          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC)] HASH:[$$17]  |PARTITIONED|
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC), $$18(ASC)] HASH:[$$17]  |PARTITIONED|
             group by ([$$17 := %0->$$1; $$18 := %0->$$3]) decor ([]) {
                       aggregate [$$20] <- [function-call: hive:max(PARTIAL1), Args:[%0->$$3]]
                       -- AGGREGATE  |LOCAL|
@@ -50,7 +50,7 @@
                }
         -- PRE_CLUSTERED_GROUP_BY[$$17]  |PARTITIONED|
           exchange 
-          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC)] HASH:[$$17]  |PARTITIONED|
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC), $$18(ASC)] HASH:[$$17]  |PARTITIONED|
             group by ([$$17 := %0->$$1; $$18 := %0->$$3]) decor ([]) {
                       aggregate [$$20] <- [function-call: hive:max(PARTIAL1), Args:[%0->$$3]]
                       -- AGGREGATE  |LOCAL|
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
index c75f9f9..1af32de 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
@@ -25,4 +25,6 @@
     public IIOManager getIOManager();
 
     public ByteBuffer allocateFrame() throws HyracksDataException;
+    
+    public void deallocateFrames(int frameCount);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java
index 0ce2346..57c9133 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java
@@ -25,7 +25,7 @@
      * @param deploymentId
      * @return
      */
-    public IJobSerializerDeserializer getJobSerializerDeerializer(DeploymentId deploymentId);
+    public IJobSerializerDeserializer getJobSerializerDeserializer(DeploymentId deploymentId);
 
     /**
      * Add a deployment with the job serializer deserializer
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java
index 35a1e8b..f2d8fff 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java
@@ -15,18 +15,18 @@
 
 package edu.uci.ics.hyracks.api.job;
 
-import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import edu.uci.ics.hyracks.api.deployment.DeploymentId;
 
 public class JobSerializerDeserializerContainer implements IJobSerializerDeserializerContainer {
 
     private IJobSerializerDeserializer defaultJobSerDe = new JobSerializerDeserializer();
-    private Map<DeploymentId, IJobSerializerDeserializer> jobSerializerDeserializerMap = new HashMap<DeploymentId, IJobSerializerDeserializer>();
+    private Map<DeploymentId, IJobSerializerDeserializer> jobSerializerDeserializerMap = new ConcurrentHashMap<DeploymentId, IJobSerializerDeserializer>();
 
     @Override
-    public synchronized IJobSerializerDeserializer getJobSerializerDeerializer(DeploymentId deploymentId) {
+    public synchronized IJobSerializerDeserializer getJobSerializerDeserializer(DeploymentId deploymentId) {
         if (deploymentId == null) {
             return defaultJobSerDe;
         }
@@ -44,4 +44,9 @@
         jobSerializerDeserializerMap.remove(deploymentId);
     }
 
+    @Override
+    public String toString() {
+        return jobSerializerDeserializerMap.toString();
+    }
+
 }
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
index aa7008f..3ea81ef 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
@@ -41,4 +41,10 @@
         return ByteBuffer.allocate(frameSize);
     }
 
+    @Override
+    public void deallocateFrames(int frameCount) {
+        // TODO Auto-generated method stub
+        
+    }
+
 }
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
index 2ca7ccf..828f2fb 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java
@@ -31,7 +31,9 @@
     protected IJobSerializerDeserializerContainer jobSerDeContainer = new JobSerializerDeserializerContainer();
     protected ThreadFactory threadFactory = new ThreadFactory() {
         public Thread newThread(Runnable r) {
-            return new Thread(r);
+            Thread t = new Thread(r);
+            t.setDaemon(true);
+            return t;
         }
     };
 
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java
index 3a35c25..9be3818 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java
@@ -31,7 +31,6 @@
  * This is the IJobSerializerDeserializer implementation for jobs with dynamic deployed jars.
  * 
  * @author yingyib
- *
  */
 public class ClassLoaderJobSerializerDeserializer implements IJobSerializerDeserializer {
 
@@ -99,4 +98,9 @@
     public ClassLoader getClassLoader() throws HyracksException {
         return classLoader;
     }
+
+    @Override
+    public String toString() {
+        return classLoader.toString();
+    }
 }
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java
index 0677e2e..0724a01 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java
@@ -89,7 +89,7 @@
      */
     public static void deploy(DeploymentId deploymentId, List<URL> urls, IJobSerializerDeserializerContainer container,
             ServerContext ctx, boolean isNC) throws HyracksException {
-        IJobSerializerDeserializer jobSerDe = container.getJobSerializerDeerializer(deploymentId);
+        IJobSerializerDeserializer jobSerDe = container.getJobSerializerDeserializer(deploymentId);
         if (jobSerDe == null) {
             jobSerDe = new ClassLoaderJobSerializerDeserializer();
             container.addJobSerializerDeserializer(deploymentId, jobSerDe);
@@ -116,7 +116,7 @@
         try {
             IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer();
             IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer
-                    .getJobSerializerDeerializer(deploymentId);
+                    .getJobSerializerDeserializer(deploymentId);
             Object obj = jobSerDe == null ? JavaSerializationUtils.deserialize(bytes) : jobSerDe.deserialize(bytes);
             return obj;
         } catch (Exception e) {
@@ -138,7 +138,7 @@
         try {
             IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer();
             IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer
-                    .getJobSerializerDeerializer(deploymentId);
+                    .getJobSerializerDeserializer(deploymentId);
             Class<?> cl = jobSerDe == null ? JavaSerializationUtils.loadClass(className) : jobSerDe
                     .loadClass(className);
             return cl;
@@ -160,7 +160,7 @@
         try {
             IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer();
             IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer
-                    .getJobSerializerDeerializer(deploymentId);
+                    .getJobSerializerDeserializer(deploymentId);
             ClassLoader cl = jobSerDe == null ? DeploymentUtils.class.getClassLoader() : jobSerDe.getClassLoader();
             return cl;
         } catch (Exception e) {
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 c72ced1..20075ff 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
@@ -115,6 +115,11 @@
     }
 
     @Override
+    public void deallocateFrames(int frameCount) {
+        joblet.deallocateFrames(frameCount);
+    }
+
+    @Override
     public int getFrameSize() {
         return joblet.getFrameSize();
     }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java
index 8add260..4829173 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/MemoryManager.java
@@ -26,10 +26,11 @@
 
     @Override
     public boolean allocate(long memory) {
-        if (this.memory.addAndGet(-memory) < 0) {
-            this.memory.addAndGet(memory);
-            return false;
-        }
+        // commented as now the deallocation is not implemented yet.
+        //        if (this.memory.addAndGet(-memory) < 0) {
+        //            this.memory.addAndGet(memory);
+        //            return false;
+        //        }
         return true;
     }
 
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 04b91b2..31cd29b 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
@@ -43,6 +43,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
 
@@ -53,8 +54,8 @@
     private final MarshalledWritable<Configuration> config;
     private final IInputSplitProviderFactory factory;
 
-    public MapperOperatorDescriptor(IOperatorDescriptorRegistry spec, int jobId, MarshalledWritable<Configuration> config,
-            IInputSplitProviderFactory factory) throws HyracksDataException {
+    public MapperOperatorDescriptor(IOperatorDescriptorRegistry spec, int jobId,
+            MarshalledWritable<Configuration> config, IInputSplitProviderFactory factory) throws HyracksDataException {
         super(spec, 0, 1);
         this.jobId = jobId;
         this.config = config;
@@ -94,7 +95,7 @@
 
             public void initBlock(int blockId) throws HyracksDataException {
                 runGen = new ExternalSortRunGenerator(ctx, new int[] { 0 }, null, comparatorFactories,
-                        helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit);
+                        helper.getMapOutputRecordDescriptorWithoutExtraFields(), Algorithm.MERGE_SORT, framesLimit);
                 this.blockId = blockId;
             }
 
@@ -114,7 +115,8 @@
                     runGen.nextFrame(frame);
                     fta.reset(frame, true);
                     if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                        throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + frame.capacity() + ")");
+                        throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
+                                + frame.capacity() + ")");
                     }
                 }
             }
@@ -224,7 +226,7 @@
                     comparators[i] = comparatorFactories[i].createBinaryComparator();
                 }
                 ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, runGen.getFrameSorter(),
-                        runGen.getRuns(), new int[] { 0 }, comparators,
+                        runGen.getRuns(), new int[] { 0 }, comparators, null,
                         helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit, delegatingWriter);
                 merger.process();
             }
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 30ba3ec..3bb78f9 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
@@ -108,7 +108,7 @@
             runs.add(rfw.createReader());
         }
         RunFileWriter rfw = new RunFileWriter(outFile, ctx.getIOManager());
-        ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators,
+        ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators, null,
                 recordDescriptor, framesLimit, rfw);
         merger.process();
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
index 235b0d0..2dda9cc 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
@@ -31,18 +32,21 @@
     private final int nSenders;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDescriptor;
     private final IPartitionBatchManager pbm;
 
     private RunMergingFrameReader merger;
 
     public SortMergeFrameReader(IHyracksTaskContext ctx, int maxConcurrentMerges, int nSenders, int[] sortFields,
-            IBinaryComparator[] comparators, RecordDescriptor recordDescriptor, IPartitionBatchManager pbm) {
+            IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDescriptor,
+            IPartitionBatchManager pbm) {
         this.ctx = ctx;
         this.maxConcurrentMerges = maxConcurrentMerges;
         this.nSenders = nSenders;
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDescriptor = recordDescriptor;
         this.pbm = pbm;
     }
@@ -57,7 +61,7 @@
             List<IFrameReader> batch = new ArrayList<IFrameReader>();
             pbm.getNextBatch(batch, nSenders);
             merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames, sortFields,
-                    comparators, recordDescriptor);
+                    comparators, nmkComputer, recordDescriptor);
         } else {
             // multi level merge.
             throw new HyracksDataException("Not yet supported");
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
index 32269bd..1cf402b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -23,6 +23,8 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -39,19 +41,23 @@
     private final ITuplePartitionComputerFactory tpcf;
     private final int[] sortFields;
     private final IBinaryComparatorFactory[] comparatorFactories;
+    private final INormalizedKeyComputerFactory nkcFactory;
     private final boolean stable;
 
-    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
-            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
-        this(spec, tpcf, sortFields, comparatorFactories, false);
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITuplePartitionComputerFactory tpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+            INormalizedKeyComputerFactory nkcFactory) {
+        this(spec, tpcf, sortFields, comparatorFactories, nkcFactory, false);
     }
 
-    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
-            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, boolean stable) {
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITuplePartitionComputerFactory tpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+            INormalizedKeyComputerFactory nkcFactory, boolean stable) {
         super(spec);
         this.tpcf = tpcf;
         this.sortFields = sortFields;
         this.comparatorFactories = comparatorFactories;
+        this.nkcFactory = nkcFactory;
         this.stable = stable;
     }
 
@@ -71,9 +77,10 @@
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
+        INormalizedKeyComputer nmkComputer = nkcFactory == null ? null : nkcFactory.createNormalizedKeyComputer();
         IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
         IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
-                sortFields, comparators, recordDesc, pbm);
+                sortFields, comparators, nmkComputer, recordDesc, pbm);
         BitSet expectedPartitions = new BitSet();
         expectedPartitions.set(0, nProducerPartitions);
         return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
new file mode 100644
index 0000000..48ee410
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AbstractAccumulatingAggregatorDescriptorFactory.java
@@ -0,0 +1,41 @@
+/*
+ * 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.group;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractAccumulatingAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory#createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext, edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int[], int[], edu.uci.ics.hyracks.api.comm.IFrameWriter)
+     */
+    @Override
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, int[] keyFields, int[] keyFieldsInPartialResults, IFrameWriter writer)
+            throws HyracksDataException {
+        return this
+                .createAggregator(ctx, inRecordDescriptor, outRecordDescriptor, keyFields, keyFieldsInPartialResults);
+    }
+
+    abstract protected IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx,
+            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int[] keyFields,
+            final int[] keyFieldsInPartialResults) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AbstractRunningAggregatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AbstractRunningAggregatorDescriptor.java
new file mode 100644
index 0000000..3dcd4c3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AbstractRunningAggregatorDescriptor.java
@@ -0,0 +1,41 @@
+/*
+ * 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.group;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+public abstract class AbstractRunningAggregatorDescriptor implements IAggregatorDescriptor {
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor#outputPartialResult(edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder, edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor, int, edu.uci.ics.hyracks.dataflow.std.group.AggregateState)
+     */
+    @Override
+    public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            AggregateState state) throws HyracksDataException {
+        return false;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor#outputFinalResult(edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder, edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor, int, edu.uci.ics.hyracks.dataflow.std.group.AggregateState)
+     */
+    @Override
+    public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            AggregateState state) throws HyracksDataException {
+        return false;
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index ecd5284..df74a93 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -38,9 +38,6 @@
 import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
 import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
 
-/**
- *
- */
 public class HashSpillableTableFactory implements ISpillableTableFactory {
 
     private static final long serialVersionUID = 1L;
@@ -104,7 +101,7 @@
         }
 
         final IAggregatorDescriptor aggregator = aggregateFactory.createAggregator(ctx, inRecordDescriptor,
-                outRecordDescriptor, keyFields, keyFieldsInPartialResults);
+                outRecordDescriptor, keyFields, keyFieldsInPartialResults, null);
 
         final AggregateState aggregateState = aggregator.createAggregateStates();
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java
index afe460c..f9d4cd3 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java
@@ -18,9 +18,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 
-/**
- *
- */
 public interface IAggregatorDescriptor {
 
     /**
@@ -82,9 +79,10 @@
      * @param offset
      * @param state
      *            The aggregation state.
+     * @return TODO
      * @throws HyracksDataException
      */
-    public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+    public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
             AggregateState state) throws HyracksDataException;
 
     /**
@@ -97,9 +95,10 @@
      * @param offset
      * @param state
      *            The aggregation state.
+     * @return TODO
      * @throws HyracksDataException
      */
-    public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+    public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
             AggregateState state) throws HyracksDataException;
 
     public void close();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
index a4c0564..241956b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
@@ -16,17 +16,15 @@
 
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
-/**
- *
- */
 public interface IAggregatorDescriptorFactory extends Serializable {
 
     IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, int[] keyFields, final int[] keyFieldsInPartialResults)
-            throws HyracksDataException;
+            RecordDescriptor outRecordDescriptor, int[] keyFields, final int[] keyFieldsInPartialResults,
+            IFrameWriter writer) throws HyracksDataException;
 
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
index 239f24f..08b2218 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
@@ -21,16 +21,14 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.std.group.AbstractAccumulatingAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 
-/**
- *
- */
-public class MultiFieldsAggregatorFactory implements IAggregatorDescriptorFactory {
+public class MultiFieldsAggregatorFactory extends AbstractAccumulatingAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
     private final IFieldAggregateDescriptorFactory[] aggregatorFactories;
@@ -78,8 +76,8 @@
             }
 
             @Override
-            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
-                    AggregateState state) throws HyracksDataException {
+            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor,
+                    int tIndex, AggregateState state) throws HyracksDataException {
                 DataOutput dos = tupleBuilder.getDataOutput();
 
                 int tupleOffset = accessor.getTupleStartOffset(tIndex);
@@ -90,11 +88,11 @@
                             ((AggregateState[]) state.state)[i]);
                     tupleBuilder.addFieldEndOffset();
                 }
-
+                return true;
             }
 
             @Override
-            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 DataOutput dos = tupleBuilder.getDataOutput();
 
@@ -110,6 +108,7 @@
                     }
                     tupleBuilder.addFieldEndOffset();
                 }
+                return true;
             }
 
             @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
index 21b3b60..fdb59a4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
@@ -25,6 +25,8 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -47,6 +49,7 @@
     private final Object stateId;
     private final int[] keyFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final AggregateState aggregateState;
     private final ArrayTupleBuilder tupleBuilder;
     private final int[] storedKeys;
@@ -76,7 +79,7 @@
     private final FrameTupleAccessor outFrameAccessor;
 
     ExternalGroupMergeOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
-            IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nmkFactory, int[] keyFields,
             IAggregatorDescriptorFactory mergerFactory, boolean isOutputSorted, int framesLimit,
             RecordDescriptor outRecordDescriptor) throws HyracksDataException {
         this.stateId = stateId;
@@ -85,13 +88,14 @@
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
+        this.nmkComputer = nmkFactory == null ? null : nmkFactory.createNormalizedKeyComputer();
         int[] keyFieldsInPartialResults = new int[keyFields.length];
         for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
             keyFieldsInPartialResults[i] = i;
         }
 
         aggregator = mergerFactory.createAggregator(ctx, outRecordDescriptor, outRecordDescriptor, keyFields,
-                keyFieldsInPartialResults);
+                keyFieldsInPartialResults, writer);
         aggregateState = aggregator.createAggregateStates();
 
         storedKeys = new int[keyFields.length];
@@ -181,7 +185,7 @@
             FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
             Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
             ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), outRecordDescriptor,
-                    runNumber, comparator);
+                    runNumber, comparator, keyFields, nmkComputer);
             /**
              * current tuple index in each run
              */
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
index b914702..9296ffa 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
@@ -130,8 +130,8 @@
                 IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
                 throws HyracksDataException {
             return new ExternalGroupMergeOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
-                    AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, keyFields, mergerFactory, isOutputSorted,
-                    framesLimit, recordDescriptors[0]);
+                    AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, firstNormalizerFactory, keyFields,
+                    mergerFactory, isOutputSorted, framesLimit, recordDescriptors[0]);
         }
 
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
index f8ba003..3c0eb2b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
@@ -121,7 +121,7 @@
         }
 
         this.aggregator = aggregatorFactory.createAggregator(ctx, inRecordDescriptor, outRecordDescriptor, fields,
-                keyFieldsInPartialResults);
+                keyFieldsInPartialResults, null);
 
         this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
         accumulatorSize = 0;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
index 35cb9e1..71af928 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
@@ -55,7 +55,7 @@
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
         final IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, inRecordDescriptor,
-                outRecordDescriptor, groupFields, groupFields);
+                outRecordDescriptor, groupFields, groupFields, writer);
         final ByteBuffer copyFrame = ctx.allocateFrame();
         final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
         copyFrameAccessor.reset(copyFrame);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
index 7773765..ba9ff49 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
@@ -44,6 +44,8 @@
 
     private boolean first;
 
+    private boolean isFailed = false;
+
     public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
             IAggregatorDescriptor aggregator, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
             IFrameWriter writer) throws HyracksDataException {
@@ -119,9 +121,9 @@
         for (int j = 0; j < groupFields.length; j++) {
             tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, groupFields[j]);
         }
-        aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor, lastTupleIndex, aggregateState);
+        boolean hasOutput = aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor, lastTupleIndex, aggregateState);
 
-        if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+        if (hasOutput && !appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
                 tupleBuilder.getSize())) {
             FrameUtils.flushFrame(outFrame, writer);
             appender.reset(outFrame, true);
@@ -149,17 +151,19 @@
 
     @Override
     public void fail() throws HyracksDataException {
+        isFailed = true;
         writer.fail();
     }
 
     @Override
     public void close() throws HyracksDataException {
-        if (!first) {
+        if (!isFailed && !first) {
             writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1);
             if (appender.getTupleCount() > 0) {
                 FrameUtils.flushFrame(outFrame, writer);
             }
         }
+        aggregator.close();
         aggregateState.close();
         writer.close();
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java
new file mode 100644
index 0000000..5c96bd0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java
@@ -0,0 +1,21 @@
+/*
+ * 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.sort;
+
+public enum Algorithm {
+    QUICK_SORT,
+    MERGE_SORT
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index f80a82c..e1315e7 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -51,6 +52,15 @@
     private final IBinaryComparatorFactory[] comparatorFactories;
     private final int framesLimit;
 
+    private Algorithm alg = Algorithm.MERGE_SORT;
+
+    public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, Algorithm alg) {
+        this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+        this.alg = alg;
+    }
+
     public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
         this(spec, framesLimit, sortFields, null, comparatorFactories, recordDescriptor);
@@ -86,7 +96,7 @@
 
     public static class SortTaskState extends AbstractStateObject {
         private List<IFrameReader> runs;
-        private FrameSorter frameSorter;
+        private IFrameSorter frameSorter;
 
         public SortTaskState() {
         }
@@ -122,7 +132,7 @@
                 @Override
                 public void open() throws HyracksDataException {
                     runGen = new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
-                            comparatorFactories, recordDescriptors[0], framesLimit);
+                            comparatorFactories, recordDescriptors[0], alg, framesLimit);
                     runGen.open();
                 }
 
@@ -166,14 +176,16 @@
                     SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
                             SORT_ACTIVITY_ID), partition));
                     List<IFrameReader> runs = state.runs;
-                    FrameSorter frameSorter = state.frameSorter;
+                    IFrameSorter frameSorter = state.frameSorter;
                     IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
                     for (int i = 0; i < comparatorFactories.length; ++i) {
                         comparators[i] = comparatorFactories[i].createBinaryComparator();
                     }
+                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
+                            : firstKeyNormalizerFactory.createNormalizedKeyComputer();
                     int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
                     ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
-                            comparators, recordDescriptors[0], necessaryFrames, writer);
+                            comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
                     merger.process();
                 }
             };
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index b149e30..3736fca 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -30,15 +30,21 @@
 
 public class ExternalSortRunGenerator implements IFrameWriter {
     private final IHyracksTaskContext ctx;
-    private final FrameSorter frameSorter;
+    private final IFrameSorter frameSorter;
     private final List<IFrameReader> runs;
     private final int maxSortFrames;
 
     public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDesc, int framesLimit) throws HyracksDataException {
+            RecordDescriptor recordDesc, Algorithm alg, int framesLimit) throws HyracksDataException {
         this.ctx = ctx;
-        frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
+        if (alg == Algorithm.MERGE_SORT) {
+            frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    recordDesc);
+        } else {
+            frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    recordDesc);
+        }
         runs = new LinkedList<IFrameReader>();
         maxSortFrames = framesLimit - 1;
     }
@@ -87,7 +93,7 @@
     public void fail() throws HyracksDataException {
     }
 
-    public FrameSorter getFrameSorter() {
+    public IFrameSorter getFrameSorter() {
         return frameSorter;
     }
 
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 510dfd6..eaf4162 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
@@ -23,6 +23,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.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -59,6 +60,7 @@
     private final List<IFrameReader> runs;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDesc;
     private final int framesLimit;
     private final IFrameWriter writer;
@@ -66,8 +68,8 @@
     private ByteBuffer outFrame;
     private FrameTupleAppender outFrameAppender;
 
-    private FrameSorter frameSorter; // Used in External sort, no replacement
-                                     // selection
+    private IFrameSorter frameSorter; // Used in External sort, no replacement
+                                      // selection
     private FrameTupleAccessor outFrameAccessor; // Used in External sort, with
                                                  // replacement selection
     private final int outputLimit; // Used in External sort, with replacement
@@ -76,14 +78,15 @@
                              // selection and limit on output size
 
     // Constructor for external sort, no replacement selection
-    public ExternalSortRunMerger(IHyracksTaskContext ctx, FrameSorter frameSorter, List<IFrameReader> runs,
-            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit,
-            IFrameWriter writer) {
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+            int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+            RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
         this.ctx = ctx;
         this.frameSorter = frameSorter;
         this.runs = new LinkedList<IFrameReader>(runs);
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
         this.framesLimit = framesLimit;
         this.writer = writer;
@@ -92,11 +95,13 @@
 
     // Constructor for external sort with replacement selection
     public ExternalSortRunMerger(IHyracksTaskContext ctx, int outputLimit, List<IFrameReader> runs, int[] sortFields,
-            IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
+            IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc,
+            int framesLimit, IFrameWriter writer) {
         this.ctx = ctx;
         this.runs = new LinkedList<IFrameReader>(runs);
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
         this.framesLimit = framesLimit;
         this.writer = writer;
@@ -162,7 +167,7 @@
 
     private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
         RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields, comparators,
-                recordDesc);
+                nmkComputer, recordDesc);
         merger.open();
         try {
             while (merger.nextFrame(outFrame)) {
@@ -263,7 +268,7 @@
                 runCursors[i] = runs.get(i);
             }
             RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields,
-                    comparators, recordDesc);
+                    comparators, nmkComputer, recordDesc);
             merger.open();
             try {
                 while (merger.nextFrame(outFrame)) {
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/FrameSorterMergeSort.java
similarity index 97%
rename from hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
rename to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
index a6bb4e2..cc0f1ef 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/FrameSorterMergeSort.java
@@ -31,7 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
-public class FrameSorter {
+public class FrameSorterMergeSort implements IFrameSorter {
     private final IHyracksTaskContext ctx;
     private final int[] sortFields;
     private final INormalizedKeyComputer nkc;
@@ -50,7 +50,7 @@
     private int[] tPointersTemp;
     private int tupleCount;
 
-    public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
+    public FrameSorterMergeSort(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
         this.ctx = ctx;
@@ -69,15 +69,18 @@
         dataFrameCount = 0;
     }
 
+    @Override
     public void reset() {
         dataFrameCount = 0;
         tupleCount = 0;
     }
 
+    @Override
     public int getFrameCount() {
         return dataFrameCount;
     }
 
+    @Override
     public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
         ByteBuffer copyFrame;
         if (dataFrameCount == buffers.size()) {
@@ -90,6 +93,7 @@
         ++dataFrameCount;
     }
 
+    @Override
     public void sortFrames() {
         int nBuffers = dataFrameCount;
         tupleCount = 0;
@@ -123,6 +127,7 @@
         }
     }
 
+    @Override
     public void flushFrames(IFrameWriter writer) throws HyracksDataException {
         appender.reset(outFrame, true);
         for (int ptr = 0; ptr < tupleCount; ++ptr) {
@@ -241,6 +246,7 @@
         return 0;
     }
 
+    @Override
     public void close() {
         this.buffers.clear();
     }
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/FrameSorterQuickSort.java
similarity index 71%
copy from hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
copy to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
index a6bb4e2..083f4a7 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/FrameSorterQuickSort.java
@@ -29,9 +29,8 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
-public class FrameSorter {
+public class FrameSorterQuickSort implements IFrameSorter {
     private final IHyracksTaskContext ctx;
     private final int[] sortFields;
     private final INormalizedKeyComputer nkc;
@@ -47,10 +46,9 @@
 
     private int dataFrameCount;
     private int[] tPointers;
-    private int[] tPointersTemp;
     private int tupleCount;
 
-    public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
+    public FrameSorterQuickSort(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
         this.ctx = ctx;
@@ -69,15 +67,18 @@
         dataFrameCount = 0;
     }
 
+    @Override
     public void reset() {
         dataFrameCount = 0;
         tupleCount = 0;
     }
 
+    @Override
     public int getFrameCount() {
         return dataFrameCount;
     }
 
+    @Override
     public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
         ByteBuffer copyFrame;
         if (dataFrameCount == buffers.size()) {
@@ -90,6 +91,7 @@
         ++dataFrameCount;
     }
 
+    @Override
     public void sortFrames() {
         int nBuffers = dataFrameCount;
         tupleCount = 0;
@@ -118,11 +120,11 @@
             }
         }
         if (tupleCount > 0) {
-            tPointersTemp = new int[tPointers.length];
-            sort(0, tupleCount);
+            sort(tPointers, 0, tupleCount);
         }
     }
 
+    @Override
     public void flushFrames(IFrameWriter writer) throws HyracksDataException {
         appender.reset(outFrame, true);
         for (int ptr = 0; ptr < tupleCount; ++ptr) {
@@ -145,73 +147,75 @@
         }
     }
 
-    private void sort(int offset, int length) {
-        int step = 1;
-        int len = length;
-        int end = offset + len;
-        /** bottom-up merge */
-        while (step < len) {
-            /** merge */
-            for (int i = offset; i < end; i += 2 * step) {
-                int next = i + step;
-                if (next < end) {
-                    merge(i, next, step, Math.min(step, end - next));
-                } else {
-                    System.arraycopy(tPointers, i * 4, tPointersTemp, i * 4, (end - i) * 4);
+    private void sort(int[] tPointers, int offset, int length) {
+        int m = offset + (length >> 1);
+        int mi = tPointers[m * 4];
+        int mj = tPointers[m * 4 + 1];
+        int mv = tPointers[m * 4 + 3];
+
+        int a = offset;
+        int b = a;
+        int c = offset + length - 1;
+        int d = c;
+        while (true) {
+            while (b <= c) {
+                int cmp = compare(tPointers, b, mi, mj, mv);
+                if (cmp > 0) {
+                    break;
                 }
+                if (cmp == 0) {
+                    swap(tPointers, a++, b);
+                }
+                ++b;
             }
-            /** prepare next phase merge */
-            step *= 2;
-            int[] tmp = tPointersTemp;
-            tPointersTemp = tPointers;
-            tPointers = tmp;
-        }
-    }
-
-    /** Merge two subarrays into one */
-    private void merge(int start1, int start2, int len1, int len2) {
-        int targetPos = start1;
-        int pos1 = start1;
-        int pos2 = start2;
-        int end1 = start1 + len1 - 1;
-        int end2 = start2 + len2 - 1;
-        while (pos1 <= end1 && pos2 <= end2) {
-            int cmp = compare(pos1, pos2);
-            if (cmp <= 0) {
-                copy(pos1, targetPos);
-                pos1++;
-            } else {
-                copy(pos2, targetPos);
-                pos2++;
+            while (c >= b) {
+                int cmp = compare(tPointers, c, mi, mj, mv);
+                if (cmp < 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, c, d--);
+                }
+                --c;
             }
-            targetPos++;
+            if (b > c)
+                break;
+            swap(tPointers, b++, c--);
         }
-        if (pos1 <= end1) {
-            int rest = end1 - pos1 + 1;
-            System.arraycopy(tPointers, pos1 * 4, tPointersTemp, targetPos * 4, rest * 4);
+
+        int s;
+        int n = offset + length;
+        s = Math.min(a - offset, b - a);
+        vecswap(tPointers, offset, b - s, s);
+        s = Math.min(d - c, n - d - 1);
+        vecswap(tPointers, b, n - s, s);
+
+        if ((s = b - a) > 1) {
+            sort(tPointers, offset, s);
         }
-        if (pos2 <= end2) {
-            int rest = end2 - pos2 + 1;
-            System.arraycopy(tPointers, pos2 * 4, tPointersTemp, targetPos * 4, rest * 4);
+        if ((s = d - c) > 1) {
+            sort(tPointers, n - s, s);
         }
     }
 
-    private void copy(int src, int dest) {
-        tPointersTemp[dest * 4] = tPointers[src * 4];
-        tPointersTemp[dest * 4 + 1] = tPointers[src * 4 + 1];
-        tPointersTemp[dest * 4 + 2] = tPointers[src * 4 + 2];
-        tPointersTemp[dest * 4 + 3] = tPointers[src * 4 + 3];
+    private void swap(int x[], int a, int b) {
+        for (int i = 0; i < 4; ++i) {
+            int t = x[a * 4 + i];
+            x[a * 4 + i] = x[b * 4 + i];
+            x[b * 4 + i] = t;
+        }
     }
 
-    private int compare(int tp1, int tp2) {
+    private void vecswap(int x[], int a, int b, int n) {
+        for (int i = 0; i < n; i++, a++, b++) {
+            swap(x, a, b);
+        }
+    }
+
+    private int compare(int[] tPointers, int tp1, int tp2i, int tp2j, int tp2v) {
         int i1 = tPointers[tp1 * 4];
         int j1 = tPointers[tp1 * 4 + 1];
         int v1 = tPointers[tp1 * 4 + 3];
-
-        int tp2i = tPointers[tp2 * 4];
-        int tp2j = tPointers[tp2 * 4 + 1];
-        int tp2v = tPointers[tp2 * 4 + 3];
-
         if (v1 != tp2v) {
             return ((((long) v1) & 0xffffffffL) < (((long) tp2v) & 0xffffffffL)) ? -1 : 1;
         }
@@ -225,12 +229,12 @@
         fta2.reset(buf2);
         for (int f = 0; f < comparators.length; ++f) {
             int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf1.array(), j1 + (fIdx - 1) * 4);
-            int f1End = IntSerDeUtils.getInt(buf1.array(), j1 + fIdx * 4);
+            int f1Start = fIdx == 0 ? 0 : buf1.getInt(j1 + (fIdx - 1) * 4);
+            int f1End = buf1.getInt(j1 + fIdx * 4);
             int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
             int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf2.array(), j2 + (fIdx - 1) * 4);
-            int f2End = IntSerDeUtils.getInt(buf2.array(), j2 + fIdx * 4);
+            int f2Start = fIdx == 0 ? 0 : buf2.getInt(j2 + (fIdx - 1) * 4);
+            int f2End = buf2.getInt(j2 + fIdx * 4);
             int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
             int l2 = f2End - f2Start;
             int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
@@ -241,7 +245,8 @@
         return 0;
     }
 
+    @Override
     public void close() {
         this.buffers.clear();
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
new file mode 100644
index 0000000..6778852
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.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.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameSorter {
+
+    public void reset();
+
+    public int getFrameCount();
+
+    public void insertFrame(ByteBuffer buffer) throws HyracksDataException;
+
+    public void sortFrames();
+
+    public void flushFrames(IFrameWriter writer) throws HyracksDataException;
+
+    public void close();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 04c82af..6fa21b5 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -77,7 +77,7 @@
     }
 
     public static class SortTaskState extends AbstractStateObject {
-        private FrameSorter frameSorter;
+        private FrameSorterMergeSort frameSorter;
 
         public SortTaskState() {
         }
@@ -111,7 +111,7 @@
                 @Override
                 public void open() throws HyracksDataException {
                     state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
-                    state.frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory,
+                    state.frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory,
                             comparatorFactories, recordDescriptors[0]);
                     state.frameSorter.reset();
                 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
index bff2e21..ef1ae88 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -201,9 +202,11 @@
                         comparators[i] = comparatorFactories[i].createBinaryComparator();
                     }
 
+                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
+                            : firstKeyNormalizerFactory.createNormalizedKeyComputer();
                     int necessaryFrames = Math.min(runs.size() + 2, memSize);
                     ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, outputLimit, runs, sortFields,
-                            comparators, recordDescriptors[0], necessaryFrames, writer);
+                            comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
 
                     merger.processWithReplacementSelection();
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
index 16b3c12..00fbe9b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -35,6 +36,7 @@
     private final List<ByteBuffer> inFrames;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDesc;
     private final FrameTupleAppender outFrameAppender;
     private ReferencedPriorityQueue topTuples;
@@ -42,12 +44,14 @@
     private FrameTupleAccessor[] tupleAccessors;
 
     public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
-            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc) {
+            int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+            RecordDescriptor recordDesc) {
         this.ctx = ctx;
         this.runCursors = runCursors;
         this.inFrames = inFrames;
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
         outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
     }
@@ -56,7 +60,8 @@
     public void open() throws HyracksDataException {
         tupleAccessors = new FrameTupleAccessor[runCursors.length];
         Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
-        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator);
+        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator,
+                sortFields, nmkComputer);
         tupleIndexes = new int[runCursors.length];
         for (int i = 0; i < runCursors.length; i++) {
             tupleIndexes[i] = 0;
@@ -143,21 +148,25 @@
     private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
         return new Comparator<ReferenceEntry>() {
             public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+                int nmk1 = tp1.getNormalizedKey();
+                int nmk2 = tp1.getNormalizedKey();
+                if (nmk1 > nmk2) {
+                    return 1;
+                }
+                if (nmk1 < nmk2) {
+                    return -1;
+                }
+
                 FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
                 FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
-                int j1 = tp1.getTupleIndex();
-                int j2 = tp2.getTupleIndex();
                 byte[] b1 = fta1.getBuffer().array();
                 byte[] b2 = fta2.getBuffer().array();
+                int[] tPointers1 = tp1.getTPointers();
+                int[] tPointers2 = tp2.getTPointers();
+
                 for (int f = 0; f < sortFields.length; ++f) {
-                    int fIdx = sortFields[f];
-                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
-                            + fta1.getFieldStartOffset(j1, fIdx);
-                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
-                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
-                            + fta2.getFieldStartOffset(j2, fIdx);
-                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
-                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                    int c = comparators[f].compare(b1, tPointers1[2 * f + 1], tPointers1[2 * f + 2], b2,
+                            tPointers2[2 * f + 1], tPointers2[2 * f + 2]);
                     if (c != 0) {
                         return c;
                     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
index c1efa91..c06b50c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
@@ -14,18 +14,24 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.util;
 
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class ReferenceEntry {
     private final int runid;
     private FrameTupleAccessor acccessor;
     private int tupleIndex;
+    private int[] tPointers;
 
-    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex) {
+    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+            INormalizedKeyComputer nmkComputer) {
         super();
         this.runid = runid;
         this.acccessor = fta;
-        this.tupleIndex = tupleIndex;
+        this.tPointers = new int[1 + 2 * keyFields.length];
+        if (fta != null) {
+            initTPointer(fta, tupleIndex, keyFields, nmkComputer);
+        }
     }
 
     public int getRunid() {
@@ -40,11 +46,38 @@
         this.acccessor = fta;
     }
 
+    public int[] getTPointers() {
+        return tPointers;
+    }
+
     public int getTupleIndex() {
         return tupleIndex;
     }
 
-    public void setTupleIndex(int tupleIndex) {
+    public int getNormalizedKey() {
+        return tPointers[0];
+    }
+
+    public void setTupleIndex(int tupleIndex, int[] keyFields, INormalizedKeyComputer nmkComputer) {
+        initTPointer(acccessor, tupleIndex, keyFields, nmkComputer);
+    }
+
+    private void initTPointer(FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+            INormalizedKeyComputer nmkComputer) {
         this.tupleIndex = tupleIndex;
+        byte[] b1 = fta.getBuffer().array();
+        for (int f = 0; f < keyFields.length; ++f) {
+            int fIdx = keyFields[f];
+            tPointers[2 * f + 1] = fta.getTupleStartOffset(tupleIndex) + fta.getFieldSlotsLength()
+                    + fta.getFieldStartOffset(tupleIndex, fIdx);
+            tPointers[2 * f + 2] = fta.getFieldEndOffset(tupleIndex, fIdx) - fta.getFieldStartOffset(tupleIndex, fIdx);
+            if (f == 0) {
+                if (nmkComputer != null) {
+                    tPointers[0] = nmkComputer.normalize(b1, tPointers[1], tPointers[2]);
+                } else {
+                    tPointers[0] = 0;
+                }
+            }
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
index 7767ace..225f583 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -18,6 +18,7 @@
 import java.util.BitSet;
 import java.util.Comparator;
 
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
@@ -30,21 +31,25 @@
     private int nItems;
 
     private final Comparator<ReferenceEntry> comparator;
+    private final INormalizedKeyComputer nmkComputer;
+    private final int[] keyFields;
 
     public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
-            Comparator<ReferenceEntry> comparator) {
+            Comparator<ReferenceEntry> comparator, int[] keyFields, INormalizedKeyComputer nmkComputer) {
         this.frameSize = frameSize;
         this.recordDescriptor = recordDescriptor;
         if (initSize < 1)
             throw new IllegalArgumentException();
         this.comparator = comparator;
+        this.nmkComputer = nmkComputer;
+        this.keyFields = keyFields;
         nItems = initSize;
         size = (initSize + 1) & 0xfffffffe;
         entries = new ReferenceEntry[size];
         runAvail = new BitSet(size);
         runAvail.set(0, initSize, true);
         for (int i = 0; i < size; i++) {
-            entries[i] = new ReferenceEntry(i, null, -1);
+            entries[i] = new ReferenceEntry(i, null, -1, keyFields, nmkComputer);
         }
     }
 
@@ -71,7 +76,7 @@
             entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
         }
         entry.getAccessor().reset(fta.getBuffer());
-        entry.setTupleIndex(tIndex);
+        entry.setTupleIndex(tIndex, keyFields, nmkComputer);
 
         add(entry);
         return entry.getRunid();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index 380ba29..a2ef99a 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -93,8 +94,8 @@
                         PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
                         new IBinaryComparatorFactory[] {
                                 PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
-                printer, 0);
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                        new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
 
         runTest(spec);
     }
@@ -148,8 +149,8 @@
                         PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
                         new IBinaryComparatorFactory[] {
                                 PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0, filter,
-                0);
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                        new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, filter, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), filter, 0, printer, 0);
 
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 48db176..faa55e8 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -89,7 +90,8 @@
                         new int[] { 1 }, new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }), new int[] { 1 },
                         new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) }), sorter, 0, printer, 0);
+                                .of(UTF8StringPointable.FACTORY) }, new UTF8StringNormalizedKeyComputerFactory()),
+                sorter, 0, printer, 0);
 
         runTest(spec);
     }
@@ -138,8 +140,8 @@
                         PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
                         new IBinaryComparatorFactory[] {
                                 PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
-                printer, 0);
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                        new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
 
         runTest(spec);
     }
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
index c61c15b..61c5a1c 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
@@ -156,8 +156,8 @@
         jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
         jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
-                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
-                0, writeOperator, 0);
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, null),
+                sortOperator, 0, writeOperator, 0);
         jobSpec.addRoot(writeOperator);
 
         IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
index 3e43fb1..baa4dc7 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
@@ -163,8 +163,8 @@
         jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
         jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
-                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
-                0, writeOperator, 0);
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, null),
+                sortOperator, 0, writeOperator, 0);
         jobSpec.addRoot(writeOperator);
 
         IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
index fe25db8..19200ee 100644
--- a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
+++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java
@@ -133,7 +133,7 @@
         metaPage.getBuffer().putInt(NUM_HASHES_USED_OFFSET, 0);
         metaPage.getBuffer().putLong(NUM_ELEMENTS_OFFSET, 0L);
         metaPage.getBuffer().putLong(NUM_BITS_OFFSET, 0L);
-        metaPage.releaseWriteLatch();
+        metaPage.releaseWriteLatch(true);
         bufferCache.unpin(metaPage);
         bufferCache.closeFile(fileId);
     }
@@ -249,7 +249,7 @@
             metaPage.getBuffer().putInt(NUM_HASHES_USED_OFFSET, numHashes);
             metaPage.getBuffer().putLong(NUM_ELEMENTS_OFFSET, numElements);
             metaPage.getBuffer().putLong(NUM_BITS_OFFSET, numBits);
-            metaPage.releaseWriteLatch();
+            metaPage.releaseWriteLatch(true);
             bufferCache.unpin(metaPage);
         }
 
@@ -273,7 +273,7 @@
         public void end() throws HyracksDataException, IndexException {
             for (int i = 0; i < numPages; ++i) {
                 ICachedPage page = bloomFilterPages.get(i);
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(true);
             }
         }
 
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 67392ac..cdaf144 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
@@ -185,45 +185,50 @@
         ByteBuffer right = rightFrame.getBuffer();
         int tupleCount = getTupleCount();
 
-        // Find split point, and determine into which frame the new tuple should be inserted into.
-        int tuplesToLeft;
+        // Find split point, and determine into which frame the new tuple should be inserted into.  
         ITreeIndexFrame targetFrame = null;
-
-        int totalSize = 0;
-        int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
-        int i;
-        for (i = 0; i < tupleCount; ++i) {
-            frameTuple.resetByTupleIndex(this, i);
-            totalSize += tupleWriter.bytesRequired(frameTuple) + childPtrSize + slotManager.getSlotSize();
-            if (totalSize >= halfPageSize) {
-                break;
-            }
-        }
-
+        frameTuple.resetByTupleIndex(this, tupleCount - 1);
+        int tuplesToLeft;
         if (cmp.compare(tuple, frameTuple) > 0) {
-            tuplesToLeft = i;
+            // This is a special optimization case when the tuple to be inserted is the largest key on the page.
             targetFrame = rightFrame;
+            tuplesToLeft = tupleCount;
         } else {
-            tuplesToLeft = i + 1;
-            targetFrame = this;
+            int totalSize = 0;
+            int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
+            int i;
+            for (i = 0; i < tupleCount; ++i) {
+                frameTuple.resetByTupleIndex(this, i);
+                totalSize += tupleWriter.bytesRequired(frameTuple) + childPtrSize + slotManager.getSlotSize();
+                if (totalSize >= halfPageSize) {
+                    break;
+                }
+            }
+
+            if (cmp.compare(tuple, frameTuple) > 0) {
+                tuplesToLeft = i;
+                targetFrame = rightFrame;
+            } else {
+                tuplesToLeft = i + 1;
+                targetFrame = this;
+            }
+            int tuplesToRight = tupleCount - tuplesToLeft;
+
+            // Copy entire page.
+            System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity());
+
+            // On the right page we need to copy rightmost slots to left.
+            int src = rightFrame.getSlotManager().getSlotEndOff();
+            int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft
+                    * rightFrame.getSlotManager().getSlotSize();
+            int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight;
+            System.arraycopy(right.array(), src, right.array(), dest, length);
+            right.putInt(tupleCountOff, tuplesToRight);
+
+            // On the left page, remove the highest key and make its child pointer
+            // the rightmost child pointer.
+            buf.putInt(tupleCountOff, tuplesToLeft);
         }
-        int tuplesToRight = tupleCount - tuplesToLeft;
-
-        // Copy entire page.
-        System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity());
-
-        // On the right page we need to copy rightmost slots to left.
-        int src = rightFrame.getSlotManager().getSlotEndOff();
-        int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft
-                * rightFrame.getSlotManager().getSlotSize();
-        int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight;
-        System.arraycopy(right.array(), src, right.array(), dest, length);
-        right.putInt(tupleCountOff, tuplesToRight);
-
-        // On the left page, remove the highest key and make its child pointer
-        // the rightmost child pointer.
-        buf.putInt(tupleCountOff, tuplesToLeft);
-
         // Copy the split key to be inserted.
         // We must do so because setting the new split key will overwrite the
         // old split key, and we cannot insert the existing split key at this point.
@@ -245,7 +250,6 @@
         // Compact both pages.
         rightFrame.compact();
         compact();
-
         // Insert the saved split key.
         int targetTupleIndex;
         // it's safe to catch this exception since it will have been caught before reaching here
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 187cd52..995e8a7 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
@@ -49,7 +49,7 @@
     public int getBytesRequriedToWriteTuple(ITupleReference tuple) {
         return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize();
     }
-    
+
     @Override
     public void initBuffer(byte level) {
         super.initBuffer(level);
@@ -148,45 +148,51 @@
 
         // Find split point, and determine into which frame the new tuple should
         // be inserted into.
-        int tuplesToLeft;
         ITreeIndexFrame targetFrame = null;
-        int totalSize = 0;
-        int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
-        int i;
-        for (i = 0; i < tupleCount; ++i) {
-            frameTuple.resetByTupleIndex(this, i);
-            totalSize += tupleWriter.getCopySpaceRequired(frameTuple) + slotManager.getSlotSize();
-            if (totalSize >= halfPageSize) {
-                break;
-            }
-        }
-
-        if (cmp.compare(tuple, frameTuple) >= 0) {
-            tuplesToLeft = i + 1;
+        frameTuple.resetByTupleIndex(this, tupleCount - 1);
+        if (cmp.compare(tuple, frameTuple) > 0) {
+            // This is a special optimization case when the tuple to be inserted is the largest key on the page.
             targetFrame = rightFrame;
         } else {
-            tuplesToLeft = i;
-            targetFrame = this;
+            int tuplesToLeft;
+            int totalSize = 0;
+            int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2;
+            int i;
+            for (i = 0; i < tupleCount; ++i) {
+                frameTuple.resetByTupleIndex(this, i);
+                totalSize += tupleWriter.getCopySpaceRequired(frameTuple) + slotManager.getSlotSize();
+                if (totalSize >= halfPageSize) {
+                    break;
+                }
+            }
+
+            if (cmp.compare(tuple, frameTuple) >= 0) {
+                tuplesToLeft = i + 1;
+                targetFrame = rightFrame;
+            } else {
+                tuplesToLeft = i;
+                targetFrame = this;
+            }
+            int tuplesToRight = tupleCount - tuplesToLeft;
+
+            // Copy entire page.
+            System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity());
+
+            // On the right page we need to copy rightmost slots to the left.
+            int src = rightFrame.getSlotManager().getSlotEndOff();
+            int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft
+                    * rightFrame.getSlotManager().getSlotSize();
+            int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight;
+            System.arraycopy(right.array(), src, right.array(), dest, length);
+            right.putInt(tupleCountOff, tuplesToRight);
+
+            // On left page only change the tupleCount indicator.
+            buf.putInt(tupleCountOff, tuplesToLeft);
+
+            // Compact both pages.
+            rightFrame.compact();
+            compact();
         }
-        int tuplesToRight = tupleCount - tuplesToLeft;
-
-        // Copy entire page.
-        System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity());
-
-        // On the right page we need to copy rightmost slots to the left.
-        int src = rightFrame.getSlotManager().getSlotEndOff();
-        int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft
-                * rightFrame.getSlotManager().getSlotSize();
-        int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight;
-        System.arraycopy(right.array(), src, right.array(), dest, length);
-        right.putInt(tupleCountOff, tuplesToRight);
-
-        // On left page only change the tupleCount indicator.
-        buf.putInt(tupleCountOff, tuplesToLeft);
-
-        // Compact both pages.
-        rightFrame.compact();
-        compact();
 
         // Insert the new tuple.
         int targetTupleIndex;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
index 2071a77..2316c6b 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java
@@ -27,19 +27,29 @@
     @Override
     public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference frameTuple, MultiComparator multiCmp,
             FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy) {
-        if (frame.getTupleCount() <= 0) {
+        int tupleCount = frame.getTupleCount();
+        if (tupleCount <= 0) {
             return GREATEST_KEY_INDICATOR;
         }
 
         int mid;
-        int begin = 0;
-        int end = frame.getTupleCount() - 1;
+        int begin;
+        int end = tupleCount - 1;
+
+        frameTuple.resetByTupleIndex(frame, end);
+        int cmp = multiCmp.compare(searchKey, frameTuple);
+        if (cmp > 0) {
+            // This is a special optimization case when the tuple to be searched is larger than all the keys on the page.
+            begin = tupleCount;
+        } else {
+            begin = 0;
+        }
 
         while (begin <= end) {
             mid = (begin + end) / 2;
             frameTuple.resetByTupleIndex(frame, mid);
 
-            int cmp = multiCmp.compare(searchKey, frameTuple);
+            cmp = multiCmp.compare(searchKey, frameTuple);
             if (cmp < 0) {
                 end = mid - 1;
             } else if (cmp > 0) {
@@ -66,7 +76,7 @@
         }
 
         if (matchPolicy == FindTupleNoExactMatchPolicy.HIGHER_KEY) {
-            if (begin > frame.getTupleCount() - 1) {
+            if (begin > tupleCount - 1) {
                 return GREATEST_KEY_INDICATOR;
             }
             frameTuple.resetByTupleIndex(frame, begin);
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 ff94040..fef0f81 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
@@ -217,7 +217,7 @@
                 ctx.interiorFrame.setPage(smPage);
                 ctx.interiorFrame.setSmFlag(false);
             } finally {
-                smPage.releaseWriteLatch();
+                smPage.releaseWriteLatch(true);
                 bufferCache.unpin(smPage);
             }
         }
@@ -261,11 +261,11 @@
                 int targetTupleIndex = ctx.interiorFrame.findInsertTupleIndex(ctx.splitKey.getTuple());
                 ctx.interiorFrame.insert(ctx.splitKey.getTuple(), targetTupleIndex);
             } finally {
-                newLeftNode.releaseWriteLatch();
+                newLeftNode.releaseWriteLatch(true);
                 bufferCache.unpin(newLeftNode);
             }
         } finally {
-            leftNode.releaseWriteLatch();
+            leftNode.releaseWriteLatch(true);
             bufferCache.unpin(leftNode);
         }
     }
@@ -447,7 +447,7 @@
             treeLatch.writeLock().unlock();
             throw e;
         } finally {
-            rightNode.releaseWriteLatch();
+            rightNode.releaseWriteLatch(true);
             bufferCache.unpin(rightNode);
         }
         return false;
@@ -533,7 +533,7 @@
 
                     ctx.splitKey.setPages(pageId, rightPageId);
                 } finally {
-                    rightNode.releaseWriteLatch();
+                    rightNode.releaseWriteLatch(true);
                     bufferCache.unpin(rightNode);
                 }
                 break;
@@ -616,7 +616,7 @@
                 if (parentIsReadLatched) {
                     parent.releaseReadLatch();
                 } else {
-                    parent.releaseWriteLatch();
+                    parent.releaseWriteLatch(true);
                 }
                 bufferCache.unpin(parent);
             }
@@ -664,7 +664,7 @@
                                         // Insert or update op. Both can cause split keys to propagate upwards. 
                                         insertInterior(interiorNode, pageId, ctx.splitKey.getTuple(), ctx);
                                     } finally {
-                                        interiorNode.releaseWriteLatch();
+                                        interiorNode.releaseWriteLatch(true);
                                         bufferCache.unpin(interiorNode);
                                     }
                                 } else {
@@ -694,7 +694,7 @@
                     if (isReadLatched) {
                         node.releaseReadLatch();
                     } else {
-                        node.releaseWriteLatch();
+                        node.releaseWriteLatch(true);
                     }
                     bufferCache.unpin(node);
 
@@ -747,7 +747,7 @@
                     }
                 }
                 if (ctx.op != IndexOperation.SEARCH) {
-                    node.releaseWriteLatch();
+                    node.releaseWriteLatch(true);
                     bufferCache.unpin(node);
                 }
                 if (restartOp) {
@@ -764,7 +764,7 @@
                     if (isReadLatched) {
                         node.releaseReadLatch();
                     } else {
-                        node.releaseWriteLatch();
+                        node.releaseWriteLatch(true);
                     }
                     bufferCache.unpin(node);
                     ctx.exceptionHandled = true;
@@ -777,7 +777,7 @@
                 if (isReadLatched) {
                     node.releaseReadLatch();
                 } else {
-                    node.releaseWriteLatch();
+                    node.releaseWriteLatch(true);
                 }
                 bufferCache.unpin(node);
             }
@@ -897,9 +897,9 @@
         }
 
         @Override
-        public ITreeIndexCursor createSearchCursor() {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
-            return new BTreeRangeSearchCursor(leafFrame, false);
+            return new BTreeRangeSearchCursor(leafFrame, exclusive);
         }
 
         @Override
@@ -990,7 +990,7 @@
                     leafFrontier.pageId = freePageManager.getFreePage(metaFrame);
 
                     ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId);
-                    leafFrontier.page.releaseWriteLatch();
+                    leafFrontier.page.releaseWriteLatch(true);
                     bufferCache.unpin(leafFrontier.page);
 
                     splitKey.setRightPage(leafFrontier.pageId);
@@ -1062,7 +1062,7 @@
 
                 ((IBTreeInteriorFrame) interiorFrame).deleteGreatest();
 
-                frontier.page.releaseWriteLatch();
+                frontier.page.releaseWriteLatch(true);
                 bufferCache.unpin(frontier.page);
                 frontier.pageId = freePageManager.getFreePage(metaFrame);
 
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
index ee65a46..38c1624 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
@@ -51,6 +51,7 @@
     private final IBTreeLeafFrame frame;
     private final ITreeIndexTupleReference frameTuple;
     private final boolean exclusiveLatchNodes;
+    private boolean isPageDirty;
 
     private RangePredicate pred;
     private MultiComparator lowKeyCmp;
@@ -61,8 +62,8 @@
     // For storing the count.
     private byte[] countBuf = new byte[4];
     private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
-    private ArrayTupleReference countTuple = new ArrayTupleReference();    
-    
+    private ArrayTupleReference countTuple = new ArrayTupleReference();
+
     public BTreeCountingSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
         this.frame = frame;
         this.frameTuple = frame.createTupleReference();
@@ -74,7 +75,7 @@
         // in case open is called multiple times without closing
         if (page != null) {
             if (exclusiveLatchNodes) {
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(isPageDirty);
             } else {
                 page.releaseReadLatch();
             }
@@ -82,6 +83,7 @@
         }
 
         page = ((BTreeCursorInitialState) initialState).getPage();
+        isPageDirty = false;
         frame.setPage(page);
 
         pred = (RangePredicate) searchPred;
@@ -107,7 +109,7 @@
         }
 
         tupleIndex = getLowKeyIndex();
-        stopTupleIndex = getHighKeyIndex();        
+        stopTupleIndex = getHighKeyIndex();
     }
 
     private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
@@ -115,13 +117,14 @@
             ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
             if (exclusiveLatchNodes) {
                 nextLeaf.acquireWriteLatch();
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(isPageDirty);
             } else {
                 nextLeaf.acquireReadLatch();
                 page.releaseReadLatch();
             }
             bufferCache.unpin(page);
             page = nextLeaf;
+            isPageDirty = false;
             frame.setPage(page);
             nextLeafPage = frame.getNextLeaf();
         } while (frame.getTupleCount() == 0 && nextLeafPage > 0);
@@ -199,7 +202,7 @@
     public void close() throws HyracksDataException {
         if (page != null) {
             if (exclusiveLatchNodes) {
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(isPageDirty);
             } else {
                 page.releaseReadLatch();
             }
@@ -208,6 +211,7 @@
         tupleBuilder.reset();
         tupleIndex = 0;
         page = null;
+        isPageDirty = false;
         pred = null;
         count = -1;
     }
@@ -218,7 +222,7 @@
             close();
         } catch (Exception e) {
             e.printStackTrace();
-        }        
+        }
     }
 
     @Override
@@ -246,4 +250,13 @@
         return exclusiveLatchNodes;
     }
 
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        if (exclusiveLatchNodes) {
+            isPageDirty = true;
+        } else {
+            throw new HyracksDataException("This cursor has not been created with the intention to allow updates.");
+        }
+    }
+
 }
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 4546b0e..584af4e 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -40,6 +40,7 @@
     private final IBTreeLeafFrame frame;
     private final ITreeIndexTupleReference frameTuple;
     private final boolean exclusiveLatchNodes;
+    private boolean isPageDirty;
 
     private IBufferCache bufferCache = null;
     private int fileId = -1;
@@ -80,7 +81,7 @@
     public void close() throws HyracksDataException {
         if (page != null) {
             if (exclusiveLatchNodes) {
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(isPageDirty);
             } else {
                 page.releaseReadLatch();
             }
@@ -89,6 +90,7 @@
 
         tupleIndex = 0;
         page = null;
+        isPageDirty = false;
         pred = null;
     }
 
@@ -114,7 +116,7 @@
             ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
             if (exclusiveLatchNodes) {
                 nextLeaf.acquireWriteLatch();
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(isPageDirty);
             } else {
                 nextLeaf.acquireReadLatch();
                 page.releaseReadLatch();
@@ -122,6 +124,7 @@
             bufferCache.unpin(page);
 
             page = nextLeaf;
+            isPageDirty = false;
             frame.setPage(page);
             pageId = nextLeafPage;
             nextLeafPage = frame.getNextLeaf();
@@ -163,12 +166,13 @@
 
                 // unlatch/unpin
                 if (exclusiveLatchNodes) {
-                    page.releaseWriteLatch();
+                    page.releaseWriteLatch(isPageDirty);
                 } else {
                     page.releaseReadLatch();
                 }
                 bufferCache.unpin(page);
                 page = null;
+                isPageDirty = false;
 
                 // reconcile
                 searchCb.reconcile(reconciliationTuple);
@@ -240,7 +244,7 @@
         // in case open is called multiple times without closing
         if (page != null) {
             if (exclusiveLatchNodes) {
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(isPageDirty);
             } else {
                 page.releaseReadLatch();
             }
@@ -251,6 +255,7 @@
         originalKeyCmp = initialState.getOriginalKeyComparator();
         pageId = ((BTreeCursorInitialState) initialState).getPageId();
         page = initialState.getPage();
+        isPageDirty = false;
         frame.setPage(page);
 
         pred = (RangePredicate) searchPred;
@@ -300,4 +305,13 @@
     public boolean exclusiveLatchNodes() {
         return exclusiveLatchNodes;
     }
+
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        if (exclusiveLatchNodes) {
+            isPageDirty = true;
+        } else {
+            throw new HyracksDataException("This cursor has not been created with the intention to allow updates.");
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java
index 5eb3cdd..a9aaad8 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java
@@ -83,7 +83,7 @@
      * Creates a cursor appropriate for passing into search().
      * 
      */
-    public IIndexCursor createSearchCursor();
+    public IIndexCursor createSearchCursor(boolean exclusive);
 
     /**
      * Open the given cursor for an index search using the given predicate as
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
index 5102d27..a097425 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
@@ -15,17 +15,20 @@
 
 package edu.uci.ics.hyracks.storage.am.common.api;
 
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
 public interface ITreeIndexCursor extends IIndexCursor {
 
-	public ICachedPage getPage();
+    public ICachedPage getPage();
 
-	public void setBufferCache(IBufferCache bufferCache);
+    public void setBufferCache(IBufferCache bufferCache);
 
-	public void setFileId(int fileId);
+    public void setFileId(int fileId);
 
-	// For allowing updates.
-	public boolean exclusiveLatchNodes();
+    // For allowing updates.
+    public boolean exclusiveLatchNodes();
+
+    public void markCurrentTupleAsUpdated() throws HyracksDataException;
 }
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 83fb5ee..2696af9 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
@@ -70,7 +70,7 @@
     protected abstract void resetSearchPredicate(int tupleIndex);
 
     protected IIndexCursor createCursor() {
-        return indexAccessor.createSearchCursor();
+        return indexAccessor.createSearchCursor(false);
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
index 6e17ff8..9c2268d 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
@@ -23,21 +23,20 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 
 /**
- * Quick & dirty data generator for multi-thread testing. 
- *
+ * Quick & dirty data generator for multi-thread testing.
  */
 @SuppressWarnings("rawtypes")
 public class DataGenThread extends Thread {
     public final BlockingQueue<TupleBatch> tupleBatchQueue;
     private final int maxNumBatches;
-    private final int maxOutstandingBatches;        
+    private final int maxOutstandingBatches;
     private int numBatches = 0;
     private final Random rnd;
-    
+
     // maxOutstandingBatches pre-created tuple-batches for populating the queue.
     private TupleBatch[] tupleBatches;
     private int ringPos;
-    
+
     public DataGenThread(int numConsumers, int maxNumBatches, int batchSize, ISerializerDeserializer[] fieldSerdes,
             int payloadSize, int rndSeed, int maxOutstandingBatches, boolean sorted) {
         this.maxNumBatches = maxNumBatches;
@@ -51,7 +50,7 @@
         tupleBatchQueue = new LinkedBlockingQueue<TupleBatch>(maxOutstandingBatches);
         ringPos = 0;
     }
-    
+
     public DataGenThread(int numConsumers, int maxNumBatches, int batchSize, ISerializerDeserializer[] fieldSerdes,
             IFieldValueGenerator[] fieldGens, int rndSeed, int maxOutstandingBatches) {
         this.maxNumBatches = maxNumBatches;
@@ -64,13 +63,13 @@
         tupleBatchQueue = new LinkedBlockingQueue<TupleBatch>(maxOutstandingBatches);
         ringPos = 0;
     }
-    
+
     @Override
     public void run() {
-        while(numBatches < maxNumBatches) {
+        while (numBatches < maxNumBatches) {
             boolean added = false;
             try {
-                if (tupleBatches[ringPos].inUse.compareAndSet(false, true)) {                    
+                if (tupleBatches[ringPos].inUse.compareAndSet(false, true)) {
                     tupleBatches[ringPos].generate();
                     tupleBatchQueue.put(tupleBatches[ringPos]);
                     added = true;
@@ -89,11 +88,11 @@
             }
         }
     }
-    
+
     public TupleBatch getBatch() throws InterruptedException {
         return tupleBatchQueue.take();
     }
-    
+
     public void releaseBatch(TupleBatch batch) {
         batch.inUse.set(false);
     }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java
index d7234fa..cb9c545 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenUtils.java
@@ -23,7 +23,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 
-@SuppressWarnings("rawtypes") 
+@SuppressWarnings("rawtypes")
 public class DataGenUtils {
     public static IFieldValueGenerator getFieldGenFromSerde(ISerializerDeserializer serde, Random rnd, boolean sorted) {
         if (serde instanceof IntegerSerializerDeserializer) {
@@ -49,8 +49,9 @@
         }
         return null;
     }
-    
-    public static IFieldValueGenerator[] getFieldGensFromSerdes(ISerializerDeserializer[] serdes, Random rnd, boolean sorted) {
+
+    public static IFieldValueGenerator[] getFieldGensFromSerdes(ISerializerDeserializer[] serdes, Random rnd,
+            boolean sorted) {
         IFieldValueGenerator[] fieldValueGens = new IFieldValueGenerator[serdes.length];
         for (int i = 0; i < serdes.length; i++) {
             fieldValueGens[i] = getFieldGenFromSerde(serdes[i], rnd, sorted);
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java
index 2e25f4c..a51ce98 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java
@@ -77,14 +77,14 @@
 					metaFrame.setMaxPage(metaMaxPage);
 					metaFrame.addFreePage(freePage);
 				} finally {
-					newNode.releaseWriteLatch();
+					newNode.releaseWriteLatch(true);
 					bufferCache.unpin(newNode);
 				}
 			}
 		} catch (Exception e) {
 			e.printStackTrace();
 		} finally {
-			metaNode.releaseWriteLatch();
+			metaNode.releaseWriteLatch(true);
 			bufferCache.unpin(metaNode);
 		}
 	}
@@ -136,7 +136,7 @@
 							metaFrame.addFreePage(nextPage);
 						}
 					} finally {
-						nextNode.releaseWriteLatch();
+						nextNode.releaseWriteLatch(true);
 						bufferCache.unpin(nextNode);
 					}
 				} else {
@@ -146,7 +146,7 @@
 				}
 			}
 		} finally {
-			metaNode.releaseWriteLatch();
+			metaNode.releaseWriteLatch(true);
 			bufferCache.unpin(metaNode);
 		}
 
@@ -164,7 +164,7 @@
 			metaFrame.setPage(metaNode);
 			maxPage = metaFrame.getMaxPage();
 		} finally {
-			metaNode.releaseWriteLatch();
+			metaNode.releaseWriteLatch(true);
 			bufferCache.unpin(metaNode);
 		}
 		return maxPage;
@@ -183,7 +183,7 @@
 			metaFrame.initBuffer(META_PAGE_LEVEL_INDICATOR);
 			metaFrame.setMaxPage(currentMaxPage);
 		} finally {
-			metaNode.releaseWriteLatch();
+			metaNode.releaseWriteLatch(true);
 			bufferCache.unpin(metaNode);
 		}
 	}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index c14f23a..e191fd0 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -111,7 +111,7 @@
             frame.setPage(rootNode);
             frame.initBuffer((byte) 0);
         } finally {
-            rootNode.releaseWriteLatch();
+            rootNode.releaseWriteLatch(true);
             bufferCache.unpin(rootNode);
         }
     }
@@ -264,7 +264,7 @@
                 throw new TreeIndexException("Cannot bulk-load a non-empty tree.");
             }
 
-            this.cmp = MultiComparator.createIgnoreFieldLength(cmpFactories);
+            this.cmp = MultiComparator.create(cmpFactories);
 
             leafFrame.setMultiComparator(cmp);
             interiorFrame.setMultiComparator(cmp);
@@ -293,7 +293,7 @@
         protected void handleException() throws HyracksDataException {
             // Unlatch and unpin pages.
             for (NodeFrontier nodeFrontier : nodeFrontiers) {
-                nodeFrontier.page.releaseWriteLatch();
+                nodeFrontier.page.releaseWriteLatch(true);
                 bufferCache.unpin(nodeFrontier.page);
             }
             releasedLatches = true;
@@ -309,7 +309,7 @@
                 System.arraycopy(lastNodeFrontier.page.getBuffer().array(), 0, newRoot.getBuffer().array(), 0,
                         lastNodeFrontier.page.getBuffer().capacity());
             } finally {
-                newRoot.releaseWriteLatch();
+                newRoot.releaseWriteLatch(true);
                 bufferCache.unpin(newRoot);
 
                 // register old root as a free page
@@ -318,7 +318,7 @@
                 if (!releasedLatches) {
                     for (int i = 0; i < nodeFrontiers.size(); i++) {
                         try {
-                            nodeFrontiers.get(i).page.releaseWriteLatch();
+                            nodeFrontiers.get(i).page.releaseWriteLatch(true);
                         } catch (Exception e) {
                             //ignore illegal monitor state exception
                         }
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
index 0f97b0d..7592348 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
@@ -147,4 +147,9 @@
     public boolean exclusiveLatchNodes() {
         return false;
     }
+
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
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 3e51e20..cb4d268 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
@@ -54,7 +54,6 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
@@ -164,8 +163,7 @@
         }
 
         if (flushOnExit) {
-            BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
-                    ioOpCallback);
+            BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
             ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             accessor.scheduleFlush(cb);
             try {
@@ -370,7 +368,8 @@
         opCtx.getComponentHolder().add(flushingComponent);
         ILSMIndexAccessorInternal flushAccessor = new LSMBTreeAccessor(lsmHarness, opCtx);
         ioScheduler.scheduleOperation(new LSMBTreeFlushOperation(flushAccessor, flushingComponent, componentFileRefs
-                .getInsertIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), callback));
+                .getInsertIndexFileReference(), componentFileRefs.getBloomFilterFileReference(), callback, fileManager
+                .getBaseDir()));
     }
 
     @Override
@@ -405,7 +404,7 @@
         IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numElements,
                 bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
 
-        IIndexCursor scanCursor = accessor.createSearchCursor();
+        IIndexCursor scanCursor = accessor.createSearchCursor(false);
         accessor.search(scanCursor, nullPred);
         try {
             while (scanCursor.hasNext()) {
@@ -442,7 +441,8 @@
                 .getName(), lastFile.getFile().getName());
         ILSMIndexAccessorInternal accessor = new LSMBTreeAccessor(lsmHarness, opCtx);
         ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs
-                .getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback));
+                .getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager
+                .getBaseDir()));
     }
 
     @Override
@@ -622,7 +622,7 @@
         }
 
         @Override
-        public IIndexCursor createSearchCursor() {
+        public IIndexCursor createSearchCursor(boolean exclusive) {
             return new LSMBTreeSearchCursor(ctx);
         }
 
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
index 668e727..94b1569 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
@@ -27,21 +27,24 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
 
-public class LSMBTreeFlushOperation implements ILSMIOOperation {
+public class LSMBTreeFlushOperation implements ILSMIOOperation, Comparable<LSMBTreeFlushOperation> {
 
     private final ILSMIndexAccessorInternal accessor;
     private final ILSMComponent flushingComponent;
     private final FileReference btreeFlushTarget;
     private final FileReference bloomFilterFlushTarget;
     private final ILSMIOOperationCallback callback;
+    private final String indexIdentifier;
 
     public LSMBTreeFlushOperation(ILSMIndexAccessorInternal accessor, ILSMComponent flushingComponent,
-            FileReference btreeFlushTarget, FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback) {
+            FileReference btreeFlushTarget, FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback,
+            String indexIdentifier) {
         this.accessor = accessor;
         this.flushingComponent = flushingComponent;
         this.btreeFlushTarget = btreeFlushTarget;
         this.bloomFilterFlushTarget = bloomFilterFlushTarget;
         this.callback = callback;
+        this.indexIdentifier = indexIdentifier;
     }
 
     @Override
@@ -83,4 +86,19 @@
     public ILSMComponent getFlushingComponent() {
         return flushingComponent;
     }
+
+    @Override
+    public String getIndexUniqueIdentifier() {
+        return indexIdentifier;
+    }
+
+    @Override
+    public LSMIOOpertionType getIOOpertionType() {
+        return LSMIOOpertionType.FLUSH;
+    }
+
+    @Override
+    public int compareTo(LSMBTreeFlushOperation o) {
+        return btreeFlushTarget.getFile().getName().compareTo(o.getBTreeFlushTarget().getFile().getName());
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
index 3a608fe..a4f6875 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeMergeOperation.java
@@ -37,16 +37,18 @@
     private final FileReference btreeMergeTarget;
     private final FileReference bloomFilterMergeTarget;
     private final ILSMIOOperationCallback callback;
+    private final String indexIdentifier;
 
     public LSMBTreeMergeOperation(ILSMIndexAccessorInternal accessor, List<ILSMComponent> mergingComponents,
             ITreeIndexCursor cursor, FileReference btreeMergeTarget, FileReference bloomFilterMergeTarget,
-            ILSMIOOperationCallback callback) {
+            ILSMIOOperationCallback callback, String indexIdentifier) {
         this.accessor = accessor;
         this.mergingComponents = mergingComponents;
         this.cursor = cursor;
         this.btreeMergeTarget = btreeMergeTarget;
         this.bloomFilterMergeTarget = bloomFilterMergeTarget;
         this.callback = callback;
+        this.indexIdentifier = indexIdentifier;
     }
 
     @Override
@@ -94,4 +96,14 @@
     public List<ILSMComponent> getMergingComponents() {
         return mergingComponents;
     }
+
+    @Override
+    public String getIndexUniqueIdentifier() {
+        return indexIdentifier;
+    }
+
+    @Override
+    public LSMIOOpertionType getIOOpertionType() {
+        return LSMIOOpertionType.MERGE;
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index fc09a74..7aba7b3 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -219,4 +219,9 @@
     public boolean exclusiveLatchNodes() {
         return false;
     }
+
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
\ 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/impls/LSMBTreeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index 366fc10..dd99833 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -129,4 +129,8 @@
         return currentCursor.exclusiveLatchNodes();
     }
 
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
index 2c3940f..1638a5d 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -22,6 +22,12 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 
 public interface ILSMIOOperation extends Callable<Boolean> {
+
+    public enum LSMIOOpertionType {
+        FLUSH,
+        MERGE
+    }
+
     public Set<IODeviceHandle> getReadDevices();
 
     public Set<IODeviceHandle> getWriteDevices();
@@ -29,4 +35,8 @@
     public Boolean call() throws HyracksDataException, IndexException;
 
     public ILSMIOOperationCallback getCallback();
+
+    public String getIndexUniqueIdentifier();
+
+    public LSMIOOpertionType getIOOpertionType();
 }
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 44bcfc2..3d7512f 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
@@ -125,16 +125,25 @@
         try {
             metadataFrame.setPage(metadataPage);
             metadataFrame.setValid(true);
-
-            // Flush the single modified page to disk.
-            bufferCache.flushDirtyPage(metadataPage);
-
-            // Force modified metadata page to disk.
-            bufferCache.force(fileId, true);
         } finally {
-            metadataPage.releaseWriteLatch();
+            metadataPage.releaseWriteLatch(true);
             bufferCache.unpin(metadataPage);
         }
+
+        // WARNING: flushing the metadata page should be done after releasing the write latch; otherwise, the page 
+        // won't be flushed to disk because it won't be dirty until the write latch has been released.
+        metadataPage = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId));
+        if (metadataPage != null) {
+            try {
+                // Flush the single modified page to disk.
+                bufferCache.flushDirtyPage(metadataPage);
+            } finally {
+                bufferCache.unpin(metadataPage);
+            }
+        }
+
+        // Force modified metadata page to disk.
+        bufferCache.force(fileId, true);
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
index 25894f1..b785764 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
@@ -14,24 +14,87 @@
  */
 package edu.uci.ics.hyracks.storage.am.lsm.common.impls;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOpertionType;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 
 public class AsynchronousScheduler implements ILSMIOOperationScheduler {
+    // Since this is a asynchronous scheduler, we make sure that flush operations coming from the same lsm index
+    // will be executed serially in same order of scheduling the operations. Look at asterix issue 630.
+
     public final static AsynchronousScheduler INSTANCE = new AsynchronousScheduler();
     private ExecutorService executor;
+    private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<String, ILSMIOOperation>();
+    private final Map<String, PriorityQueue<ILSMIOOperation>> waitingFlushOperations = new HashMap<String, PriorityQueue<ILSMIOOperation>>();
 
     public void init(ThreadFactory threadFactory) {
-        executor = Executors.newCachedThreadPool(threadFactory);
+        // Creating an executor with the same configuration of Executors.newCachedThreadPool. 
+        executor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS,
+                new SynchronousQueue<Runnable>(), threadFactory) {
+
+            @Override
+            protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
+                return new LSMIOOperationTask<T>(callable);
+            }
+
+            @SuppressWarnings("unchecked")
+            @Override
+            protected void afterExecute(Runnable r, Throwable t) {
+                super.afterExecute(r, t);
+                LSMIOOperationTask<Boolean> task = (LSMIOOperationTask<Boolean>) r;
+                ILSMIOOperation executedOp = task.getOperation();
+                String id = executedOp.getIndexUniqueIdentifier();
+                synchronized (this) {
+                    runningFlushOperations.remove(id);
+                    if (waitingFlushOperations.containsKey(id)) {
+                        try {
+                            ILSMIOOperation op = waitingFlushOperations.get(id).poll();
+                            if (op != null) {
+                                scheduleOperation(op);
+                            } else {
+                                waitingFlushOperations.remove(id);
+                            }
+                        } catch (HyracksDataException e) {
+                            t = e.getCause();
+                        }
+                    }
+                }
+            }
+        };
     }
 
     @Override
     public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
-        executor.submit(operation);
+        if (operation.getIOOpertionType() == LSMIOOpertionType.MERGE) {
+            executor.submit(operation);
+        } else {
+            String id = operation.getIndexUniqueIdentifier();
+            synchronized (executor) {
+                if (runningFlushOperations.containsKey(id)) {
+                    if (waitingFlushOperations.containsKey(id)) {
+                        waitingFlushOperations.get(id).offer(operation);
+                    } else {
+                        PriorityQueue<ILSMIOOperation> q = new PriorityQueue<ILSMIOOperation>();
+                        q.offer(operation);
+                        waitingFlushOperations.put(id, q);
+                    }
+                } else {
+                    runningFlushOperations.put(id, operation);
+                    executor.submit(operation);
+                }
+            }
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIOOperationTask.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIOOperationTask.java
new file mode 100644
index 0000000..9743afe
--- /dev/null
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIOOperationTask.java
@@ -0,0 +1,33 @@
+/*
+ * 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.storage.am.lsm.common.impls;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.FutureTask;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+
+public class LSMIOOperationTask<T> extends FutureTask<T> {
+    private final ILSMIOOperation operation;
+
+    public LSMIOOperationTask(Callable<T> callable) {
+        super(callable);
+        this.operation = (ILSMIOOperation) callable;
+    }
+
+    public ILSMIOOperation getOperation() {
+        return operation;
+    }
+}
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 2bc45a9..5a4699e 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -263,4 +263,9 @@
     protected int compare(MultiComparator cmp, ITupleReference tupleA, ITupleReference tupleB) {
         return cmp.compare(tupleA, tupleB);
     }
+    
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index bd3b278..7f804c1 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -336,7 +336,7 @@
         }
 
         @Override
-        public void releaseWriteLatch() {
+        public void releaseWriteLatch(boolean markDirty) {
             latch.writeLock().unlock();
         }
 
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 0600754..7b4ae42 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
@@ -438,7 +438,7 @@
         ioScheduler.scheduleOperation(new LSMInvertedIndexFlushOperation(
                 new LSMInvertedIndexAccessor(lsmHarness, opCtx), flushingComponent, componentFileRefs
                         .getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
-                componentFileRefs.getBloomFilterFileReference(), callback));
+                componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir()));
     }
 
     @Override
@@ -458,7 +458,7 @@
                 .getInvIndex().createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
         BTreeAccessor memBTreeAccessor = memInvIndexAccessor.getBTreeAccessor();
         RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
-        IIndexCursor scanCursor = memBTreeAccessor.createSearchCursor();
+        IIndexCursor scanCursor = memBTreeAccessor.createSearchCursor(false);
         memBTreeAccessor.search(scanCursor, nullPred);
 
         // Bulk load the disk inverted index from the in-memory inverted index.
@@ -498,7 +498,7 @@
             BTree diskDeletedKeysBTree = component.getDeletedKeysBTree();
 
             // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
-            IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor();
+            IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
             deletedKeysBTreeAccessor.search(deletedKeysScanCursor, nullPred);
 
             // Bulk load the deleted-keys BTree.
@@ -543,7 +543,7 @@
         ILSMIndexAccessorInternal accessor = new LSMInvertedIndexAccessor(lsmHarness, ctx);
         ioScheduler.scheduleOperation(new LSMInvertedIndexMergeOperation(accessor, mergingComponents, cursor,
                 relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
-                relMergeFileRefs.getBloomFilterFileReference(), callback));
+                relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir()));
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
index 7e34dfe..f76085a 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -71,7 +71,7 @@
         lsmHarness.search(ctx, cursor, searchPred);
     }
 
-    public IIndexCursor createSearchCursor() {
+    public IIndexCursor createSearchCursor(boolean exclusive) {
         return new LSMInvertedIndexSearchCursor();
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
index 3efaaba..9eb317d 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
@@ -54,7 +54,7 @@
         RangePredicate btreePredicate = new RangePredicate(null, null, true, true, keyCmp, keyCmp);
         ArrayList<IIndexAccessor> btreeAccessors = lsmInitialState.getDeletedKeysBTreeAccessors();
         for (int i = 0; i < numBTrees; i++) {
-            rangeCursors[i] = btreeAccessors.get(i).createSearchCursor();
+            rangeCursors[i] = btreeAccessors.get(i).createSearchCursor(false);
             btreeAccessors.get(i).search(rangeCursors[i], btreePredicate);
         }
         setPriorityQueueComparator();
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
index 45433e7..beb7643 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexFlushOperation.java
@@ -28,23 +28,25 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
 
-public class LSMInvertedIndexFlushOperation implements ILSMIOOperation {
+public class LSMInvertedIndexFlushOperation implements ILSMIOOperation, Comparable<LSMInvertedIndexFlushOperation> {
     private final ILSMIndexAccessorInternal accessor;
     private final ILSMComponent flushingComponent;
     private final FileReference dictBTreeFlushTarget;
     private final FileReference deletedKeysBTreeFlushTarget;
     private final FileReference bloomFilterFlushTarget;
     private final ILSMIOOperationCallback callback;
+    private final String indexIdentifier;
 
     public LSMInvertedIndexFlushOperation(ILSMIndexAccessorInternal accessor, ILSMComponent flushingComponent,
             FileReference dictBTreeFlushTarget, FileReference deletedKeysBTreeFlushTarget,
-            FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback) {
+            FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback, String indexIdentifier) {
         this.accessor = accessor;
         this.flushingComponent = flushingComponent;
         this.dictBTreeFlushTarget = dictBTreeFlushTarget;
         this.deletedKeysBTreeFlushTarget = deletedKeysBTreeFlushTarget;
         this.bloomFilterFlushTarget = bloomFilterFlushTarget;
         this.callback = callback;
+        this.indexIdentifier = indexIdentifier;
     }
 
     @Override
@@ -88,4 +90,19 @@
     public ILSMComponent getFlushingComponent() {
         return flushingComponent;
     }
+
+    @Override
+    public String getIndexUniqueIdentifier() {
+        return indexIdentifier;
+    }
+
+    @Override
+    public LSMIOOpertionType getIOOpertionType() {
+        return LSMIOOpertionType.FLUSH;
+    }
+
+    @Override
+    public int compareTo(LSMInvertedIndexFlushOperation o) {
+        return dictBTreeFlushTarget.getFile().getName().compareTo(o.getDictBTreeFlushTarget().getFile().getName());
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
index 7cd921a..5a1e413 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeOperation.java
@@ -38,10 +38,11 @@
     private final FileReference deletedKeysBTreeMergeTarget;
     private final FileReference bloomFilterMergeTarget;
     private final ILSMIOOperationCallback callback;
+    private final String indexIdentifier;
 
     public LSMInvertedIndexMergeOperation(ILSMIndexAccessorInternal accessor, List<ILSMComponent> mergingComponents,
             IIndexCursor cursor, FileReference dictBTreeMergeTarget, FileReference deletedKeysBTreeMergeTarget,
-            FileReference bloomFilterMergeTarget, ILSMIOOperationCallback callback) {
+            FileReference bloomFilterMergeTarget, ILSMIOOperationCallback callback, String indexIdentifier) {
         this.accessor = accessor;
         this.mergingComponents = mergingComponents;
         this.cursor = cursor;
@@ -49,6 +50,7 @@
         this.deletedKeysBTreeMergeTarget = deletedKeysBTreeMergeTarget;
         this.bloomFilterMergeTarget = bloomFilterMergeTarget;
         this.callback = callback;
+        this.indexIdentifier = indexIdentifier;
     }
 
     @Override
@@ -103,4 +105,14 @@
     public List<ILSMComponent> getMergingComponents() {
         return mergingComponents;
     }
+
+    @Override
+    public String getIndexUniqueIdentifier() {
+        return indexIdentifier;
+    }
+
+    @Override
+    public LSMIOOpertionType getIOOpertionType() {
+        return LSMIOOpertionType.MERGE;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
index a6ff07e..48e94eb 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
@@ -77,7 +77,7 @@
                 ILSMComponent component = operationalComponents.get(i);
                 if (component.getType() == LSMComponentType.MEMORY) {
                     // No need for a bloom filter for the in-memory BTree.
-                    deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
+                    deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(false);
                 } else {
                     deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor(
                             (IBTreeLeafFrame) lsmInitState.getgetDeletedKeysBTreeLeafFrameFactory().createFrame(),
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index 19acbfb..8ceedaf 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -76,7 +76,7 @@
             ILSMComponent component = operationalComponents.get(i);
             if (component.getType() == LSMComponentType.MEMORY) {
                 // No need for a bloom filter for the in-memory BTree.
-                deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor();
+                deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(false);
             } else {
                 deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState
                         .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false,
@@ -134,7 +134,7 @@
         while (accessorIndex < indexAccessors.size()) {
             // Current cursor has been exhausted, switch to next accessor/cursor.
             currentAccessor = indexAccessors.get(accessorIndex);
-            currentCursor = currentAccessor.createSearchCursor();
+            currentCursor = currentAccessor.createSearchCursor(false);
             try {
                 currentAccessor.search(currentCursor, searchPred);
             } catch (OccurrenceThresholdPanicException e) {
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
index 7e9b483..ef8af32 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -65,7 +65,7 @@
     }
 
     @Override
-    public IIndexCursor createSearchCursor() {
+    public IIndexCursor createSearchCursor(boolean exclusive) {
         return new OnDiskInvertedIndexSearchCursor(searcher, index.getInvListTypeTraits().length);
     }
 
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
index c742d0c..84e0fad 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
@@ -67,7 +67,7 @@
         // Avoid object creation if this.btreeAccessor == btreeAccessor.
         if (this.btreeAccessor != btreeAccessor) {
             this.btreeAccessor = btreeAccessor;
-            this.btreeCursor = btreeAccessor.createSearchCursor();
+            this.btreeCursor = btreeAccessor.createSearchCursor(false);
             this.countingCursor = btreeAccessor.createCountingSearchCursor();
             this.btreePred = btreePred;
             this.btreePred.setLowKeyComparator(tokenFieldsCmp);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 67d1796..d72e019 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -325,7 +325,7 @@
         }
 
         public void pinNextPage() throws HyracksDataException {
-            currentPage.releaseWriteLatch();
+            currentPage.releaseWriteLatch(true);
             bufferCache.unpin(currentPage);
             currentPageId++;
             currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
@@ -431,7 +431,7 @@
             btreeBulkloader.end();
 
             if (currentPage != null) {
-                currentPage.releaseWriteLatch();
+                currentPage.releaseWriteLatch(true);
                 bufferCache.unpin(currentPage);
             }
             invListsMaxPageId = currentPageId;
@@ -484,7 +484,7 @@
         }
 
         @Override
-        public IIndexCursor createSearchCursor() {
+        public IIndexCursor createSearchCursor(boolean exclusive) {
             return new OnDiskInvertedIndexSearchCursor(searcher, index.getInvListTypeTraits().length);
         }
 
@@ -565,6 +565,12 @@
         public ByteBuffer allocateFrame() {
             return ByteBuffer.allocate(FRAME_SIZE);
         }
+        
+        @Override
+        public void deallocateFrames(int frameCount) {
+            // TODO Auto-generated method stub
+            
+        }
     }
 
     @Override
@@ -584,7 +590,7 @@
         // Scan the btree and validate the order of elements in each inverted-list.
         IIndexAccessor btreeAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE,
                 NoOpOperationCallback.INSTANCE);
-        IIndexCursor btreeCursor = btreeAccessor.createSearchCursor();
+        IIndexCursor btreeCursor = btreeAccessor.createSearchCursor(false);
         MultiComparator btreeCmp = MultiComparator.createIgnoreFieldLength(btree.getComparatorFactories());
         RangePredicate rangePred = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
         int[] fieldPermutation = new int[tokenTypeTraits.length];
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
index 5c47601..509409f 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
@@ -36,7 +36,7 @@
     public OnDiskInvertedIndexOpContext(BTree btree) {
         // TODO: Ignore opcallbacks for now.
         btreeAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        btreeCursor = btreeAccessor.createSearchCursor();
+        btreeCursor = btreeAccessor.createSearchCursor(false);
         searchCmp = MultiComparator.createIgnoreFieldLength(btree.getComparatorFactories());
         if (btree.getComparatorFactories().length > 1) {
             prefixSearchCmp = MultiComparator.create(btree.getComparatorFactories(), 0, 1);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
index defe29c..9cb7231 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
@@ -59,7 +59,7 @@
             fieldPermutation[i] = i;
         }
         tokenTuple = new PermutingTupleReference(fieldPermutation);
-        btreeCursor = btreeAccessor.createSearchCursor();
+        btreeCursor = btreeAccessor.createSearchCursor(false);
         concatTuple = new ConcatenatingTupleReference(2);
         invListCursor = invIndex.createInvertedListCursor();
         unpinNeeded = false;
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 662aa02..95d1a11 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
@@ -177,7 +177,7 @@
         LSMRTreeAccessor accessor = new LSMRTreeAccessor(lsmHarness, rctx);
         ioScheduler.scheduleOperation(new LSMRTreeFlushOperation(accessor, flushingComponent, componentFileRefs
                 .getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(), componentFileRefs
-                .getBloomFilterFileReference(), callback));
+                .getBloomFilterFileReference(), callback, fileManager.getBaseDir()));
     }
 
     @Override
@@ -191,7 +191,7 @@
         // scan the memory RTree
         ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree().createAccessor(
                 NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor();
+        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(false);
         SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
         memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
         LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(),
@@ -258,7 +258,7 @@
             BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement,
                     bloomFilterFalsePositiveRate);
 
-            IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor();
+            IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
             memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
             BTree diskBTree = component.getBTree();
 
@@ -295,7 +295,8 @@
         ILSMIndexAccessorInternal accessor = new LSMRTreeAccessor(lsmHarness, rctx);
         ioScheduler.scheduleOperation(new LSMRTreeMergeOperation((ILSMIndexAccessorInternal) accessor,
                 mergingComponents, cursor, relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs
-                        .getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback));
+                        .getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(), callback,
+                fileManager.getBaseDir()));
     }
 
     @Override
@@ -359,7 +360,7 @@
         }
 
         @Override
-        public ITreeIndexCursor createSearchCursor() {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             return new LSMRTreeSearchCursor(ctx);
         }
 
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index 4e322d4..32f3174 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -163,4 +163,8 @@
         return false;
     }
 
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
index 18d7a7e..0d40eb4 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeFlushOperation.java
@@ -27,7 +27,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
 
-public class LSMRTreeFlushOperation implements ILSMIOOperation {
+public class LSMRTreeFlushOperation implements ILSMIOOperation, Comparable<LSMRTreeFlushOperation> {
 
     private final ILSMIndexAccessorInternal accessor;
     private final ILSMComponent flushingComponent;
@@ -35,16 +35,18 @@
     private final FileReference btreeFlushTarget;
     private final FileReference bloomFilterFlushTarget;
     private final ILSMIOOperationCallback callback;
+    private final String indexIdentifier;
 
     public LSMRTreeFlushOperation(ILSMIndexAccessorInternal accessor, ILSMComponent flushingComponent,
             FileReference rtreeFlushTarget, FileReference btreeFlushTarget, FileReference bloomFilterFlushTarget,
-            ILSMIOOperationCallback callback) {
+            ILSMIOOperationCallback callback, String indexIdentifier) {
         this.accessor = accessor;
         this.flushingComponent = flushingComponent;
         this.rtreeFlushTarget = rtreeFlushTarget;
         this.btreeFlushTarget = btreeFlushTarget;
         this.bloomFilterFlushTarget = bloomFilterFlushTarget;
         this.callback = callback;
+        this.indexIdentifier = indexIdentifier;
     }
 
     @Override
@@ -89,4 +91,19 @@
     public ILSMComponent getFlushingComponent() {
         return flushingComponent;
     }
+
+    @Override
+    public String getIndexUniqueIdentifier() {
+        return indexIdentifier;
+    }
+
+    @Override
+    public LSMIOOpertionType getIOOpertionType() {
+        return LSMIOOpertionType.FLUSH;
+    }
+
+    @Override
+    public int compareTo(LSMRTreeFlushOperation o) {
+        return rtreeFlushTarget.getFile().getName().compareTo(o.getRTreeFlushTarget().getFile().getName());
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
index da7a2fb..ddf3ebf 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMergeOperation.java
@@ -36,10 +36,11 @@
     private final FileReference btreeMergeTarget;
     private final FileReference bloomFilterMergeTarget;
     private final ILSMIOOperationCallback callback;
+    private final String indexIdentifier;
 
     public LSMRTreeMergeOperation(ILSMIndexAccessorInternal accessor, List<ILSMComponent> mergingComponents,
             ITreeIndexCursor cursor, FileReference rtreeMergeTarget, FileReference btreeMergeTarget,
-            FileReference bloomFilterMergeTarget, ILSMIOOperationCallback callback) {
+            FileReference bloomFilterMergeTarget, ILSMIOOperationCallback callback, String indexIdentifier) {
         this.accessor = accessor;
         this.mergingComponents = mergingComponents;
         this.cursor = cursor;
@@ -47,6 +48,7 @@
         this.btreeMergeTarget = btreeMergeTarget;
         this.bloomFilterMergeTarget = bloomFilterMergeTarget;
         this.callback = callback;
+        this.indexIdentifier = indexIdentifier;
     }
 
     @Override
@@ -104,4 +106,14 @@
     public List<ILSMComponent> getMergingComponents() {
         return mergingComponents;
     }
+
+    @Override
+    public String getIndexUniqueIdentifier() {
+        return indexIdentifier;
+    }
+
+    @Override
+    public LSMIOOpertionType getIOOpertionType() {
+        return LSMIOOpertionType.MERGE;
+    }
 }
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 ec66a39..71d83f2 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
@@ -155,7 +155,7 @@
         opCtx.getComponentHolder().add(flushingComponent);
         ILSMIndexAccessorInternal accessor = new LSMRTreeWithAntiMatterTuplesAccessor(lsmHarness, opCtx);
         ioScheduler.scheduleOperation(new LSMRTreeFlushOperation(accessor, flushingComponent, relFlushFileRefs
-                .getInsertIndexFileReference(), null, null, callback));
+                .getInsertIndexFileReference(), null, null, callback, fileManager.getBaseDir()));
     }
 
     @Override
@@ -167,7 +167,7 @@
         LSMRTreeMemoryComponent flushingComponent = (LSMRTreeMemoryComponent) flushOp.getFlushingComponent();
         ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree().createAccessor(
                 NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor();
+        RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(false);
         SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
         memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
         LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(), null,
@@ -177,7 +177,7 @@
         // scan the memory BTree
         ITreeIndexAccessor memBTreeAccessor = flushingComponent.getBTree().createAccessor(
                 NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-        BTreeRangeSearchCursor btreeScanCursor = (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor();
+        BTreeRangeSearchCursor btreeScanCursor = (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(false);
         RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
         memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
 
@@ -256,7 +256,7 @@
         LSMComponentFileReferences relMergeFileRefs = getMergeTargetFileName(mergingComponents);
         ILSMIndexAccessorInternal accessor = new LSMRTreeWithAntiMatterTuplesAccessor(lsmHarness, rctx);
         ioScheduler.scheduleOperation(new LSMRTreeMergeOperation(accessor, mergingComponents, cursor, relMergeFileRefs
-                .getInsertIndexFileReference(), null, null, callback));
+                .getInsertIndexFileReference(), null, null, callback, fileManager.getBaseDir()));
     }
 
     @Override
@@ -298,7 +298,7 @@
         }
 
         @Override
-        public ITreeIndexCursor createSearchCursor() {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             return new LSMRTreeWithAntiMatterTuplesSearchCursor(ctx);
         }
 
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
index 0dd2281..aa3fad8 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
@@ -145,20 +145,21 @@
 
     @Override
     public void setBufferCache(IBufferCache bufferCache) {
-        // TODO Auto-generated method stub
 
     }
 
     @Override
     public void setFileId(int fileId) {
-        // TODO Auto-generated method stub
 
     }
 
     @Override
     public boolean exclusiveLatchNodes() {
-        // TODO Auto-generated method stub
         return false;
     }
 
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
index 03d9043..7946529 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
@@ -222,4 +222,8 @@
         return false;
     }
 
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
\ No newline at end of file
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 7bb3583..266686f 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
@@ -201,7 +201,7 @@
                 ICachedPage node = ctx.LSNUpdates.get(i);
                 ctx.interiorFrame.setPage(node);
                 ctx.interiorFrame.setPageLsn(incrementGlobalNsn());
-                node.releaseWriteLatch();
+                node.releaseWriteLatch(true);
                 bufferCache.unpin(node);
             }
         }
@@ -228,7 +228,7 @@
                         writeLatched = true;
 
                         if (!ctx.interiorFrame.isLeaf()) {
-                            node.releaseWriteLatch();
+                            node.releaseWriteLatch(true);
                             writeLatched = false;
                             bufferCache.unpin(node);
                             continue;
@@ -246,7 +246,7 @@
                     // Concurrent split detected, go back to parent and
                     // re-choose the best child
                     if (writeLatched) {
-                        node.releaseWriteLatch();
+                        node.releaseWriteLatch(true);
                         writeLatched = false;
                         bufferCache.unpin(node);
                     } else {
@@ -294,7 +294,7 @@
                         // already pointing to the best child
                         ctx.interiorFrame.enlarge(ctx.getTuple(), ctx.cmp);
 
-                        node.releaseWriteLatch();
+                        node.releaseWriteLatch(true);
                         writeLatched = false;
                         bufferCache.unpin(node);
                     } else {
@@ -303,7 +303,7 @@
                             readLatched = false;
                             bufferCache.unpin(node);
                         } else if (writeLatched) {
-                            node.releaseWriteLatch();
+                            node.releaseWriteLatch(true);
                             writeLatched = false;
                             bufferCache.unpin(node);
                         }
@@ -324,7 +324,7 @@
                     readLatched = false;
                     bufferCache.unpin(node);
                 } else if (writeLatched) {
-                    node.releaseWriteLatch();
+                    node.releaseWriteLatch(true);
                     writeLatched = false;
                     bufferCache.unpin(node);
                 }
@@ -359,7 +359,7 @@
                     } else if (isLeaf) {
                         // In case of a crash, we un-latch the interior node
                         // inside updateParentForInsert.
-                        node.releaseWriteLatch();
+                        node.releaseWriteLatch(true);
                         bufferCache.unpin(node);
                     }
                 }
@@ -384,7 +384,7 @@
                     } else if (isLeaf) {
                         // In case of a crash, we un-latch the interior node
                         // inside updateParentForInsert.
-                        node.releaseWriteLatch();
+                        node.releaseWriteLatch(true);
                         bufferCache.unpin(node);
                     }
                 }
@@ -424,12 +424,12 @@
                     } else if (isLeaf) {
                         // In case of a crash, we un-latch the interior node
                         // inside updateParentForInsert.
-                        node.releaseWriteLatch();
+                        node.releaseWriteLatch(true);
                         bufferCache.unpin(node);
-                        rightNode.releaseWriteLatch();
+                        rightNode.releaseWriteLatch(true);
                         bufferCache.unpin(rightNode);
                     } else {
-                        rightNode.releaseWriteLatch();
+                        rightNode.releaseWriteLatch(true);
                         bufferCache.unpin(rightNode);
                     }
 
@@ -469,16 +469,16 @@
                         } else if (isLeaf) {
                             // In case of a crash, we un-latch the interior node
                             // inside updateParentForInsert.
-                            node.releaseWriteLatch();
+                            node.releaseWriteLatch(true);
                             bufferCache.unpin(node);
-                            rightNode.releaseWriteLatch();
+                            rightNode.releaseWriteLatch(true);
                             bufferCache.unpin(rightNode);
-                            newLeftNode.releaseWriteLatch();
+                            newLeftNode.releaseWriteLatch(true);
                             bufferCache.unpin(newLeftNode);
                         } else {
-                            rightNode.releaseWriteLatch();
+                            rightNode.releaseWriteLatch(true);
                             bufferCache.unpin(rightNode);
-                            newLeftNode.releaseWriteLatch();
+                            newLeftNode.releaseWriteLatch(true);
                             bufferCache.unpin(newLeftNode);
                         }
                     }
@@ -508,7 +508,7 @@
                         break;
                     }
                     int rightPage = ctx.interiorFrame.getRightPage();
-                    parentNode.releaseWriteLatch();
+                    parentNode.releaseWriteLatch(true);
                     writeLatched = false;
                     bufferCache.unpin(parentNode);
 
@@ -529,7 +529,7 @@
                     ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), -1, ctx.cmp);
                 } catch (TreeIndexException e) {
                     if (writeLatched) {
-                        parentNode.releaseWriteLatch();
+                        parentNode.releaseWriteLatch(true);
                         writeLatched = false;
                         bufferCache.unpin(parentNode);
                     }
@@ -544,7 +544,7 @@
         } finally {
             if (!succeeded) {
                 if (writeLatched) {
-                    parentNode.releaseWriteLatch();
+                    parentNode.releaseWriteLatch(true);
                     writeLatched = false;
                     bufferCache.unpin(parentNode);
                 }
@@ -634,7 +634,7 @@
             try {
                 deleteTuple(tupleIndex, ctx);
             } finally {
-                ctx.leafFrame.getPage().releaseWriteLatch();
+                ctx.leafFrame.getPage().releaseWriteLatch(true);
                 bufferCache.unpin(ctx.leafFrame.getPage());
             }
         }
@@ -694,7 +694,7 @@
                         if (!ctx.leafFrame.isLeaf()) {
                             ctx.pathList.add(pageId, -1, -1);
 
-                            node.releaseWriteLatch();
+                            node.releaseWriteLatch(true);
                             writeLatched = false;
                             bufferCache.unpin(node);
                             continue;
@@ -707,7 +707,7 @@
                             if (tupleIndex == -1) {
                                 ctx.pathList.add(pageId, parentLsn, -1);
 
-                                node.releaseWriteLatch();
+                                node.releaseWriteLatch(true);
                                 writeLatched = false;
                                 bufferCache.unpin(node);
                                 continue;
@@ -732,7 +732,7 @@
                     readLatched = false;
                     bufferCache.unpin(node);
                 } else if (writeLatched) {
-                    node.releaseWriteLatch();
+                    node.releaseWriteLatch(true);
                     writeLatched = false;
                     bufferCache.unpin(node);
                 }
@@ -825,7 +825,7 @@
         }
 
         @Override
-        public ITreeIndexCursor createSearchCursor() {
+        public ITreeIndexCursor createSearchCursor(boolean exclusive) {
             return new RTreeSearchCursor((IRTreeInteriorFrame) interiorFrameFactory.createFrame(),
                     (IRTreeLeafFrame) leafFrameFactory.createFrame());
         }
@@ -907,7 +907,7 @@
 
                     leafFrontier.pageId = freePageManager.getFreePage(metaFrame);
 
-                    leafFrontier.page.releaseWriteLatch();
+                    leafFrontier.page.releaseWriteLatch(true);
                     bufferCache.unpin(leafFrontier.page);
 
                     leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId),
@@ -974,7 +974,7 @@
                 propagateBulk(level + 1, toRoot);
                 propagated = true;
 
-                frontier.page.releaseWriteLatch();
+                frontier.page.releaseWriteLatch(true);
                 bufferCache.unpin(frontier.page);
                 frontier.pageId = freePageManager.getFreePage(metaFrame);
 
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
index c71b3eb..ab82f08 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
@@ -252,4 +252,9 @@
     public boolean exclusiveLatchNodes() {
         return false;
     }
+    
+    @Override
+    public void markCurrentTupleAsUpdated() throws HyracksDataException {
+        throw new HyracksDataException("Updating tuples is not supported with this cursor.");
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index 2992dfe..4162dc2 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -474,18 +474,9 @@
             latch.readLock().lock();
         }
 
-        private void acquireWriteLatch(boolean markDirty) {
-            latch.writeLock().lock();
-            if (markDirty) {
-                if (dirty.compareAndSet(false, true)) {
-                    pinCount.incrementAndGet();
-                }
-            }
-        }
-
         @Override
         public void acquireWriteLatch() {
-            acquireWriteLatch(true);
+            latch.writeLock().lock();
         }
 
         @Override
@@ -494,7 +485,12 @@
         }
 
         @Override
-        public void releaseWriteLatch() {
+        public void releaseWriteLatch(boolean markDirty) {
+            if (markDirty) {
+                if (dirty.compareAndSet(false, true)) {
+                    pinCount.incrementAndGet();
+                }
+            }
             latch.writeLock().unlock();
         }
     }
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java
index b5bf3bb..6c86597 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java
@@ -25,5 +25,5 @@
 
     public void acquireWriteLatch();
 
-    public void releaseWriteLatch();
+    public void releaseWriteLatch(boolean markDirty);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
index 02ce7f1..4c158ce 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
@@ -88,7 +88,7 @@
         public SearchTask() throws HyracksDataException {
             this.cb = new SynchronizingSearchOperationCallback();
             this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
-            this.cursor = accessor.createSearchCursor();
+            this.cursor = accessor.createSearchCursor(false);
             this.predicate = new RangePredicate();
             this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
             this.tuple = new ArrayTupleReference();
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index 0a70185..f814829 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -735,7 +735,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Ordered Scan:");
         }
-        IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor();
+        IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor(false);
         RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
         indexAccessor.search(scanCursor, nullPred);
         try {
@@ -795,7 +795,7 @@
             String highKeyString = TupleUtils.printTuple(highKey, fieldSerdes);
             LOGGER.info("Range-Search in: [ " + lowKeyString + ", " + highKeyString + "]");
         }
-        ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+        ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
         MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, lowKey);
         MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, highKey);
         RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp);
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexSortedInsertTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexSortedInsertTest.java
new file mode 100644
index 0000000..e8b83e8
--- /dev/null
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexSortedInsertTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.storage.am.btree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+
+/**
+ * Tests the BTree insert operation with sorted stream of strings and integer fields using
+ * various numbers of key and payload fields.
+ * Each tests first fills a BTree with sorted generated tuples. We compare the
+ * following operations against expected results: 1. Point searches for all
+ * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
+ * search for composite keys).
+ */
+@SuppressWarnings("rawtypes")
+public abstract class OrderedIndexSortedInsertTest extends OrderedIndexTestDriver {
+
+    private final OrderedIndexTestUtils orderedIndexTestUtils;
+
+    public OrderedIndexSortedInsertTest(BTreeLeafFrameType[] leafFrameTypesToTest) {
+        super(leafFrameTypesToTest);
+        this.orderedIndexTestUtils = new OrderedIndexTestUtils();
+    }
+
+    @Override
+    protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType,
+            ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
+            throws Exception {
+        OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
+        // We assume all fieldSerdes are of the same type. Check the first one
+        // to determine which field types to generate.
+        if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
+            orderedIndexTestUtils.insertSortedIntTuples(ctx, numTuplesToInsert, getRandom());
+        } else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) {
+            orderedIndexTestUtils.insertSortedStringTuples(ctx, numTuplesToInsert, getRandom());
+        }
+
+        orderedIndexTestUtils.checkPointSearches(ctx);
+        orderedIndexTestUtils.checkScan(ctx);
+        orderedIndexTestUtils.checkDiskOrderScan(ctx);
+
+        orderedIndexTestUtils.checkRangeSearch(ctx, lowKey, highKey, true, true);
+        if (prefixLowKey != null && prefixHighKey != null) {
+            orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
+        }
+
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+
+    @Override
+    protected String getTestOpName() {
+        return "Insert-Sorted";
+    }
+}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
index 9de217b..a2a4ba8 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
@@ -43,6 +43,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.exceptions.TreeIndexDuplicateKeyException;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
@@ -93,7 +94,7 @@
         }
         MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), lowKey);
         MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), highKey);
-        IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor();
+        IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(false);
         RangePredicate rangePred = new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp,
                 highKeyCmp);
         ctx.getIndexAccessor().search(searchCursor, rangePred);
@@ -140,7 +141,7 @@
             LOGGER.info("Testing Point Searches On All Expected Keys.");
         }
         OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
-        IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor();
+        IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(false);
 
         ArrayTupleBuilder lowKeyBuilder = new ArrayTupleBuilder(ctx.getKeyFieldCount());
         ArrayTupleReference lowKey = new ArrayTupleReference();
@@ -180,6 +181,79 @@
     }
 
     @SuppressWarnings("unchecked")
+    public void insertSortedIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+        int fieldCount = ctx.getFieldCount();
+        int numKeyFields = ctx.getKeyFieldCount();
+        int[] fieldValues = new int[ctx.getFieldCount()];
+        int maxValue = (int) Math.ceil(Math.pow(numTuples, 1.0 / (double) numKeyFields));
+        Collection<CheckTuple> tmpCheckTuples = createCheckTuplesCollection();
+        for (int i = 0; i < numTuples; i++) {
+            // Set keys.
+            setIntKeyFields(fieldValues, numKeyFields, maxValue, rnd);
+            // Set values.
+            setIntPayloadFields(fieldValues, numKeyFields, fieldCount);
+
+            // Set expected values. (We also use these as the pre-sorted stream
+            // for ordered indexes bulk loading).
+            ctx.insertCheckTuple(createIntCheckTuple(fieldValues, ctx.getKeyFieldCount()), tmpCheckTuples);
+        }
+        insertCheckTuples(ctx, tmpCheckTuples);
+
+        // Add tmpCheckTuples to ctx check tuples for comparing searches.
+        for (CheckTuple checkTuple : tmpCheckTuples) {
+            ctx.insertCheckTuple(checkTuple, ctx.getCheckTuples());
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public void insertSortedStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+        int fieldCount = ctx.getFieldCount();
+        int numKeyFields = ctx.getKeyFieldCount();
+        String[] fieldValues = new String[fieldCount];
+        TreeSet<CheckTuple> tmpCheckTuples = new TreeSet<CheckTuple>();
+        for (int i = 0; i < numTuples; i++) {
+            // Set keys.
+            for (int j = 0; j < numKeyFields; j++) {
+                int length = (Math.abs(rnd.nextInt()) % 10) + 1;
+                fieldValues[j] = getRandomString(length, rnd);
+            }
+            // Set values.
+            for (int j = numKeyFields; j < fieldCount; j++) {
+                fieldValues[j] = getRandomString(5, rnd);
+            }
+            // Set expected values. We also use these as the pre-sorted stream
+            // for bulk loading.
+            ctx.insertCheckTuple(createStringCheckTuple(fieldValues, ctx.getKeyFieldCount()), tmpCheckTuples);
+        }
+        insertCheckTuples(ctx, tmpCheckTuples);
+
+        // Add tmpCheckTuples to ctx check tuples for comparing searches.
+        for (CheckTuple checkTuple : tmpCheckTuples) {
+            ctx.insertCheckTuple(checkTuple, ctx.getCheckTuples());
+        }
+    }
+
+    public static void insertCheckTuples(IIndexTestContext ctx, Collection<CheckTuple> checkTuples)
+            throws HyracksDataException, IndexException {
+        int fieldCount = ctx.getFieldCount();
+        int numTuples = checkTuples.size();
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+
+        int c = 1;
+        for (CheckTuple checkTuple : checkTuples) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (c % (numTuples / 10) == 0) {
+                    LOGGER.info("Inserting Tuple " + c + "/" + numTuples);
+                }
+            }
+            createTupleFromCheckTuple(checkTuple, tupleBuilder, tuple, ctx.getFieldSerdes());
+            ctx.getIndexAccessor().insert(tuple);
+            c++;
+        }
+    }
+
+    @SuppressWarnings("unchecked")
     public void insertStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
index b5742b5..2919805 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -98,7 +98,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Scan.");
         }
-        ITreeIndexCursor scanCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor();
+        ITreeIndexCursor scanCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor(false);
         ISearchPredicate nullPred = createNullSearchPredicate();
         ctx.getIndexAccessor().search(scanCursor, nullPred);
         Iterator<CheckTuple> checkIter = ctx.getCheckTuples().iterator();
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 705a976..6a519d0 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
@@ -727,7 +727,7 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Scan:");
         }
-        ITreeIndexCursor scanCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+        ITreeIndexCursor scanCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
         SearchPredicate nullPred = new SearchPredicate(null, null);
         indexAccessor.search(scanCursor, nullPred);
         try {
@@ -786,7 +786,7 @@
             String kString = TupleUtils.printTuple(key, fieldSerdes);
             LOGGER.info("Range-Search using key: " + kString);
         }
-        ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+        ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false);
         MultiComparator cmp = RTreeUtils.getSearchMultiComparator(cmpFactories, key);
         SearchPredicate rangePred = new SearchPredicate(key, cmp);
         indexAccessor.search(rangeCursor, rangePred);
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
index f93a007..af7cf14 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
@@ -67,7 +67,7 @@
         AbstractRTreeTestContext ctx = (AbstractRTreeTestContext) ictx;
         MultiComparator cmp = RTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), key);
 
-        ITreeIndexCursor searchCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor();
+        ITreeIndexCursor searchCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor(false);
         SearchPredicate searchPred = new SearchPredicate(key, cmp);
         ctx.getIndexAccessor().search(searchCursor, searchPred);
 
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index 10b5582..f18099b 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -46,6 +46,10 @@
     public ByteBuffer allocateFrame() {
         return jobletContext.allocateFrame();
     }
+    
+    @Override
+    public void deallocateFrames(int frameCount) {
+    }
 
     @Override
     public int getFrameSize() {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSortedInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSortedInsertTest.java
new file mode 100644
index 0000000..5e24c52
--- /dev/null
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSortedInsertTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.storage.am.btree;
+
+import java.util.Random;
+
+import org.junit.After;
+import org.junit.Before;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestContext;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
+
+/**
+ * Tests the BTree insert operation with sorted stream of strings and integer fields using
+ * various numbers of key and payload fields. Each tests first fills a BTree with
+ * randomly generated tuples. We compare the following operations against expected results:
+ * 1) Point searches for all tuples
+ * 2) Ordered scan
+ * 3) Disk-order scan
+ * 4) Range search (and prefix search for composite keys)
+ */
+public class BTreeSortedInsertTest extends OrderedIndexSortedInsertTest {
+
+    private final BTreeTestHarness harness = new BTreeTestHarness();
+
+    public BTreeSortedInsertTest() {
+        super(BTreeTestHarness.LEAF_FRAMES_TO_TEST);
+    }
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+            BTreeLeafFrameType leafType) throws Exception {
+        return BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(),
+                harness.getFileReference(), fieldSerdes, numKeys, leafType);
+    }
+
+    @Override
+    protected Random getRandom() {
+        return harness.getRandom();
+    }
+}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
index 29da625..5f77d68 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
@@ -154,7 +154,7 @@
                         if (LOGGER.isLoggable(Level.INFO)) {
                             LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId);
                         }
-                        plPage.page.releaseWriteLatch();
+                        plPage.page.releaseWriteLatch(true);
                     }
                 }
                 if (LOGGER.isLoggable(Level.INFO)) {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
index 22d3e6a..2d7a318 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
@@ -51,7 +51,7 @@
     @Override
     public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
         BTree.BTreeAccessor accessor = (BTree.BTreeAccessor) indexAccessor;
-        ITreeIndexCursor searchCursor = accessor.createSearchCursor();
+        ITreeIndexCursor searchCursor = accessor.createSearchCursor(false);
         ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor();
         MultiComparator cmp = accessor.getOpContext().cmp;
         RangePredicate rangePred = new RangePredicate(tuple, tuple, true, true, cmp, cmp);
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 f1a5355..afaf30b 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
@@ -97,7 +97,7 @@
         public SearchTask() throws HyracksDataException {
             this.cb = new SynchronizingSearchOperationCallback();
             this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
-            this.cursor = accessor.createSearchCursor();
+            this.cursor = accessor.createSearchCursor(false);
             this.predicate = new RangePredicate();
             this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
             this.tuple = new ArrayTupleReference();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 35ecc20..8ab5d22 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -52,7 +52,7 @@
     @Override
     public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
         LSMBTreeAccessor accessor = (LSMBTreeAccessor) indexAccessor;
-        IIndexCursor searchCursor = accessor.createSearchCursor();
+        IIndexCursor searchCursor = accessor.createSearchCursor(false);
         MultiComparator cmp = accessor.getMultiComparator();
         RangePredicate rangePred = new RangePredicate(tuple, tuple, true, true, cmp, cmp);
 
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 3591f78..fb06a7e 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
@@ -20,10 +20,12 @@
 import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
+import java.util.concurrent.ThreadFactory;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+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.control.nc.io.IOManager;
@@ -37,9 +39,9 @@
 import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -50,8 +52,8 @@
 
 public class LSMTreeRunner implements IExperimentRunner {
 
-    private static final int MAX_OPEN_FILES = 10000;
-    private static final int HYRACKS_FRAME_SIZE = 128;
+    private static final int MAX_OPEN_FILES = Integer.MAX_VALUE;
+    private static final int HYRACKS_FRAME_SIZE = 131072;
 
     protected IHyracksTaskContext ctx;
     protected IOManager ioManager;
@@ -61,7 +63,7 @@
 
     protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
     protected final static String sep = System.getProperty("file.separator");
-    protected final static String classDir = "/lsmtree/";
+    protected final static String classDir = "/tmp/lsmtree/";
     protected String onDiskDir;
     protected FileReference file;
 
@@ -71,6 +73,11 @@
     protected IBufferCache memBufferCache;
     private final int onDiskPageSize;
     private final int onDiskNumPages;
+    private final static ThreadFactory threadFactory = new ThreadFactory() {
+        public Thread newThread(Runnable r) {
+            return new Thread(r);
+        }
+    };
 
     public LSMTreeRunner(int numBatches, int inMemPageSize, int inMemNumPages, int onDiskPageSize, int onDiskNumPages,
             ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields,
@@ -93,11 +100,13 @@
         List<IVirtualBufferCache> virtualBufferCaches = new ArrayList<IVirtualBufferCache>();
         for (int i = 0; i < 2; i++) {
             IVirtualBufferCache virtualBufferCache = new VirtualBufferCache(new HeapBufferAllocator(), inMemPageSize,
-                    inMemNumPages);
+                    inMemNumPages / 2);
             virtualBufferCaches.add(virtualBufferCache);
         }
 
-        this.ioScheduler = SynchronousScheduler.INSTANCE;
+        this.ioScheduler = AsynchronousScheduler.INSTANCE;
+        AsynchronousScheduler.INSTANCE.init(threadFactory);
+
         lsmtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fmp, typeTraits, cmpFactories,
                 bloomFilterKeyFields, bloomFilterFalsePositiveRate, NoMergePolicy.INSTANCE,
                 new ThreadCountingTracker(), ioScheduler, NoOpIOOperationCallback.INSTANCE);
@@ -133,15 +142,23 @@
 
     @Override
     public void reset() throws Exception {
+        try {
+            lsmtree.deactivate();
+        } catch (HyracksDataException e) {
+            // ignore
+        }
+        try {
+            lsmtree.destroy();
+        } catch (HyracksDataException e) {
+            // ignore
+        }
+
         lsmtree.create();
+        lsmtree.activate();
     }
 
     @Override
     public void deinit() throws Exception {
-        bufferCache.closeFile(lsmtreeFileId);
-        bufferCache.close();
-        memBufferCache.closeFile(lsmtreeFileId);
-        memBufferCache.close();
     }
 
     public class LSMTreeThread extends Thread {
@@ -166,6 +183,7 @@
                         } catch (TreeIndexException e) {
                         }
                     }
+                    dataGen.releaseBatch(batch);
                 }
             } catch (Exception e) {
                 e.printStackTrace();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
index 641362e..2354550 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
@@ -31,18 +31,20 @@
     public static void main(String[] args) throws Exception {
         // Disable logging so we can better see the output times.
         Enumeration<String> loggers = LogManager.getLogManager().getLoggerNames();
-        while(loggers.hasMoreElements()) {
+        while (loggers.hasMoreElements()) {
             String loggerName = loggers.nextElement();
             Logger logger = LogManager.getLogManager().getLogger(loggerName);
             logger.setLevel(Level.OFF);
         }
-        
-        int numTuples = 100000; // 100K
+        boolean sorted = Boolean.parseBoolean(args[0]);
+        int numThreads =  Integer.parseInt(args[1]);
+
+        //int numTuples = 100000; // 100K
         //int numTuples = 1000000; // 1M
         //int numTuples = 2000000; // 2M
         //int numTuples = 3000000; // 3M
         //int numTuples = 10000000; // 10M
-        //int numTuples = 20000000; // 20M
+        int numTuples = 20000000; // 20M
         //int numTuples = 30000000; // 30M
         //int numTuples = 40000000; // 40M
         //int numTuples = 60000000; // 60M
@@ -50,27 +52,41 @@
         //int numTuples = 200000000; // 200M
         int batchSize = 10000;
         int numBatches = numTuples / batchSize;
-        
+
+        int payLoadSize = 240;
         ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE };
-        ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, 30);
-        
-        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, fieldSerdes.length);
-        
+        ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes, payLoadSize);
+
+        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
+                fieldSerdes.length);
+        int[] bloomFilterKeyFields = new int[cmpFactories.length];
+        for (int i = 0; i < bloomFilterKeyFields.length; i++) {
+            bloomFilterKeyFields[i] = i;
+        }
+        double bloomFilterFalsePositiveRate = 0.01;
+
         //int repeats = 1000;
         int repeats = 1;
         long[] times = new long[repeats];
 
-        int numThreads = 2;
+//        int numThreads = 4;
+//        boolean sorted = true;
         for (int i = 0; i < repeats; i++) {
             //ConcurrentSkipListRunner runner = new ConcurrentSkipListRunner(numBatches, batchSize, tupleSize, typeTraits, cmp);
-            InMemoryBTreeRunner runner = new InMemoryBTreeRunner(numBatches, 8192, 100000, typeTraits, cmpFactories);
+            //InMemoryBTreeRunner runner = new InMemoryBTreeRunner(numBatches, 8192, 100000, typeTraits, cmpFactories);
             //BTreeBulkLoadRunner runner = new BTreeBulkLoadRunner(numBatches, 8192, 100000, typeTraits, cmp, 1.0f);
-        	//BTreeRunner runner = new BTreeRunner(numBatches, 8192, 100000, typeTraits, cmp);
-        	//String btreeName = "071211";
-        	//BTreeSearchRunner runner = new BTreeSearchRunner(btreeName, 10, numBatches, 8192, 25000, typeTraits, cmp);
-        	//LSMTreeRunner runner = new LSMTreeRunner(numBatches, 8192, 100, 8192, 250, typeTraits, cmp);
-        	//LSMTreeSearchRunner runner = new LSMTreeSearchRunner(100000, numBatches, 8192, 24750, 8192, 250, typeTraits, cmp); 
-            DataGenThread dataGen = new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, 30, 50, 10, false);
+            //BTreeRunner runner = new BTreeRunner(numBatches, 8192, 100000, typeTraits, cmp);
+            //String btreeName = "071211";
+            //BTreeSearchRunner runner = new BTreeSearchRunner(btreeName, 10, numBatches, 8192, 25000, typeTraits, cmp);
+            //LSMTreeRunner runner = new LSMTreeRunner(numBatches, 8192, 100, 8192, 250, typeTraits, cmp);
+            //LSMTreeSearchRunner runner = new LSMTreeSearchRunner(100000, numBatches, 8192, 24750, 8192, 250, typeTraits, cmp); 
+            int inMemPageSize = 131072; // 128kb
+            int onDiskPageSize = inMemPageSize;
+            int inMemNumPages = 8192; // 1GB
+            int onDiskNumPages = 16384; // 2GB
+            LSMTreeRunner runner = new LSMTreeRunner(numBatches, inMemPageSize, inMemNumPages, onDiskPageSize,
+                    onDiskNumPages, typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate);
+            DataGenThread dataGen = new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, payLoadSize, 50, 10, sorted);
             dataGen.start();
             runner.reset();
             times[i] = runner.runExperiment(dataGen, numThreads);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 35afd09..34b3ff7 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -58,7 +58,7 @@
     @Override
     public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
         LSMInvertedIndexAccessor accessor = (LSMInvertedIndexAccessor) indexAccessor;
-        IIndexCursor searchCursor = accessor.createSearchCursor();
+        IIndexCursor searchCursor = accessor.createSearchCursor(false);
         IIndexCursor rangeSearchCursor = accessor.createRangeSearchCursor();
         RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
         IBinaryTokenizerFactory tokenizerFactory = invIndex.getTokenizerFactory();
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index c4e1ac5..58e0ace 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -510,7 +510,7 @@
             searchPred.setQueryTuple(searchDocument);
             searchPred.setQueryFieldIndex(0);
 
-            IIndexCursor resultCursor = accessor.createSearchCursor();
+            IIndexCursor resultCursor = accessor.createSearchCursor(false);
             boolean panic = false;
             try {
                 accessor.search(resultCursor, searchPred);
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
index c7a86d9..b52b518 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -50,7 +50,7 @@
     @Override
     public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
         LSMRTreeAccessor accessor = (LSMRTreeAccessor) indexAccessor;
-        ITreeIndexCursor searchCursor = accessor.createSearchCursor();
+        ITreeIndexCursor searchCursor = accessor.createSearchCursor(false);
         MultiComparator cmp = accessor.getMultiComparator();
         SearchPredicate rangePred = new SearchPredicate(tuple, cmp);
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
index 383cbc4..eef80d6 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -39,7 +39,7 @@
     @Override
     public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
         LSMRTreeWithAntiMatterTuplesAccessor accessor = (LSMRTreeWithAntiMatterTuplesAccessor) indexAccessor;
-        ITreeIndexCursor searchCursor = accessor.createSearchCursor();
+        ITreeIndexCursor searchCursor = accessor.createSearchCursor(false);
         MultiComparator cmp = accessor.getMultiComparator();
         SearchPredicate rangePred = new SearchPredicate(tuple, cmp);
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
index bc5d331..04cf044 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
@@ -49,7 +49,7 @@
     @Override
     public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException {
         RTree.RTreeAccessor accessor = (RTree.RTreeAccessor) indexAccessor;
-        IIndexCursor searchCursor = accessor.createSearchCursor();
+        IIndexCursor searchCursor = accessor.createSearchCursor(false);
         ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor();
         MultiComparator cmp = accessor.getOpContext().cmp;
         SearchPredicate rangePred = new SearchPredicate(tuple, cmp);
diff --git a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java
index 577abbb..ba1715f 100644
--- a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java
@@ -81,7 +81,7 @@
                 buf.put(Byte.MAX_VALUE);
             }
         } finally {
-            writePage.releaseWriteLatch();
+            writePage.releaseWriteLatch(true);
             bufferCache.unpin(writePage);
         }
         bufferCache.closeFile(firstFileId);
diff --git a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
index c67386e..118a27c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
@@ -91,7 +91,7 @@
             bufferCache.unpin(page2);
 
         } finally {
-            page.releaseWriteLatch();
+            page.releaseWriteLatch(true);
             bufferCache.unpin(page);
         }
 
@@ -228,7 +228,7 @@
                 }
                 pageContents.put(fileId, values);
             } finally {
-                page.releaseWriteLatch();
+                page.releaseWriteLatch(true);
                 bufferCache.unpin(page);
             }
         }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
index 26cb8d0..a42b411 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java
@@ -258,8 +258,9 @@
     @Override
     final public void readFields(DataInput in) throws IOException {
         reset();
-        if (vertexId == null)
+        if (vertexId == null) {
             vertexId = BspUtils.<I> createVertexIndex(getContext().getConfiguration());
+        }
         vertexId.readFields(in);
         delegate.setVertexId(vertexId);
         boolean hasVertexValue = in.readBoolean();
@@ -576,15 +577,6 @@
         return context;
     }
 
-    /**
-     * Pregelix internal use only
-     * 
-     * @param context
-     */
-    public static final void setContext(TaskAttemptContext context) {
-        Vertex.context = context;
-    }
-
     @Override
     public int hashCode() {
         return vertexId.hashCode();
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 6549c52..15aa7c4 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
@@ -182,6 +182,15 @@
     }
 
     /**
+     * Set whether the vertex state length is fixed
+     * 
+     * @param jobId
+     */
+    final public void setFixedVertexValueSize(boolean fixedSize) {
+        getConfiguration().setBoolean(INCREASE_STATE_LENGTH, !fixedSize);
+    }
+
+    /**
      * Set the frame size for a job
      * 
      * @param frameSize
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 4ee1deb..e79d4d2 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
@@ -37,11 +37,6 @@
  * them.
  */
 public class BspUtils {
-    private static Configuration defaultConf = null;
-
-    public static void setDefaultConfiguration(Configuration conf) {
-        defaultConf = conf;
-    }
 
     /**
      * Get the user's subclassed {@link VertexInputFormat}.
@@ -209,8 +204,6 @@
      */
     @SuppressWarnings("unchecked")
     public static <I extends Writable> Class<I> getVertexIndexClass(Configuration conf) {
-        if (conf == null)
-            conf = defaultConf;
         return (Class<I>) conf.getClass(PregelixJob.VERTEX_INDEX_CLASS, WritableComparable.class);
     }
 
@@ -302,8 +295,6 @@
      */
     @SuppressWarnings("unchecked")
     public static <M extends WritableSizable> Class<M> getMessageValueClass(Configuration conf) {
-        if (conf == null)
-            conf = defaultConf;
         return (Class<M>) conf.getClass(PregelixJob.MESSAGE_VALUE_CLASS, Writable.class);
     }
 
@@ -316,8 +307,6 @@
      */
     @SuppressWarnings("unchecked")
     public static <M extends Writable> Class<M> getPartialAggregateValueClass(Configuration conf) {
-        if (conf == null)
-            conf = defaultConf;
         return (Class<M>) conf.getClass(PregelixJob.PARTIAL_AGGREGATE_VALUE_CLASS, Writable.class);
     }
 
@@ -330,8 +319,6 @@
      */
     @SuppressWarnings("unchecked")
     public static <M extends Writable> Class<M> getPartialCombineValueClass(Configuration conf) {
-        if (conf == null)
-            conf = defaultConf;
         return (Class<M>) conf.getClass(PregelixJob.PARTIAL_COMBINE_VALUE_CLASS, Writable.class);
     }
 
@@ -344,8 +331,6 @@
      */
     @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);
     }
@@ -359,8 +344,6 @@
      */
     @SuppressWarnings("unchecked")
     public static <M extends Writable> Class<M> getFinalAggregateValueClass(Configuration conf) {
-        if (conf == null)
-            conf = defaultConf;
         return (Class<M>) conf.getClass(PregelixJob.FINAL_AGGREGATE_VALUE_CLASS, Writable.class);
     }
 
@@ -488,8 +471,6 @@
      */
     @SuppressWarnings({ "unchecked", "rawtypes" })
     public static <V extends VertexPartitioner> Class<V> getVertexPartitionerClass(Configuration conf) {
-        if (conf == null)
-            conf = defaultConf;
         return (Class<V>) conf.getClass(PregelixJob.PARTITIONER_CLASS, null, VertexPartitioner.class);
     }
 
@@ -502,8 +483,6 @@
      */
     @SuppressWarnings("unchecked")
     public static <V extends ICheckpointHook> Class<V> getCheckpointHookClass(Configuration conf) {
-        if (conf == null)
-            conf = defaultConf;
         return (Class<V>) conf.getClass(PregelixJob.CKP_CLASS, DefaultCheckpointHook.class, ICheckpointHook.class);
     }
 
@@ -515,7 +494,7 @@
      * @return the boolean setting of the parameter, by default it is false
      */
     public static boolean getDynamicVertexValueSize(Configuration conf) {
-        return conf.getBoolean(PregelixJob.INCREASE_STATE_LENGTH, false);
+        return conf.getBoolean(PregelixJob.INCREASE_STATE_LENGTH, true);
     }
 
     /**
diff --git a/pregelix/pregelix-benchmark/pom.xml b/pregelix/pregelix-benchmark/pom.xml
index 4d7d456..c1344ea 100644
--- a/pregelix/pregelix-benchmark/pom.xml
+++ b/pregelix/pregelix-benchmark/pom.xml
@@ -12,6 +12,42 @@
 	<artifactId>pregelix-benchmark</artifactId>
 	<name>pregelix-benchmark</name>
 	<url>http://maven.apache.org</url>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.7</source>
+					<target>1.7</target>
+					<fork>true</fork>
+				</configuration>
+			</plugin>
+
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<version>2.2-beta-5</version>
+				<configuration>
+					<descriptorRefs>
+						<descriptorRef>jar-with-dependencies</descriptorRef>
+					</descriptorRefs>
+				</configuration>
+				<executions>
+					<execution>
+						<id>make-my-jar-with-dependencies</id>
+						<phase>package</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
+
 	<dependencies>
 		<dependency>
 			<groupId>junit</groupId>
@@ -26,6 +62,12 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
+			<groupId>org.apache.giraph</groupId>
+			<artifactId>giraph-examples</artifactId>
+			<version>1.0.0</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-hdfs-core</artifactId>
 			<version>0.2.10-SNAPSHOT</version>
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
deleted file mode 100644
index 04c29de..0000000
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.benchmark;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.VLongWritable;
-
-/**
- * Demonstrates the basic Pregel PageRank implementation.
- */
-public class PageRankVertex extends Vertex<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
-
-    public static final String ITERATIONS = "HyracksPageRankVertex.iteration";
-    private final DoubleWritable vertexValue = new DoubleWritable();
-    private final DoubleWritable msg = new DoubleWritable();
-    private int maxIteration = -1;
-
-    @Override
-    public void compute(Iterable<DoubleWritable> msgIterator) {
-        if (maxIteration < 0) {
-            maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
-        }
-        if (getSuperstep() == 1) {
-            vertexValue.set(1.0 / getTotalNumVertices());
-        }
-        if (getSuperstep() >= 2 && getSuperstep() <= maxIteration) {
-            double sum = 0;
-            for (DoubleWritable msg : msgIterator) {
-                sum += msg.get();
-            }
-            vertexValue.set((0.15 / getTotalNumVertices()) + 0.85 * sum);
-        }
-
-        if (getSuperstep() >= 1 && getSuperstep() < maxIteration) {
-            long edges = getNumEdges();
-            msg.set(vertexValue.get() / edges);
-            sendMessageToAllEdges(msg);
-        } else {
-            voteToHalt();
-        }
-    }
-
-}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
similarity index 60%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
index 3d85f66..b290907 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
@@ -12,7 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -24,14 +25,13 @@
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextCCInputFormat extends TextVertexInputFormat<LongWritable, LongWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,25 +39,25 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
+            protected LongWritable getId(Text line) throws IOException {
                 items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
-            protected DoubleWritable getValue(Text line) throws IOException {
+            protected LongWritable getValue(Text line) throws IOException {
                 return null;
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
                     edge.setEntry(entry);
                     edge.setValue(null);
                     edges.add(edge);
@@ -67,4 +67,5 @@
 
         };
     }
+
 }
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java
new file mode 100644
index 0000000..770c6e1
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * 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.benchmark.io;
+
+import java.io.IOException;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextCCOutputFormat extends TextVertexOutputFormat<LongWritable, LongWritable, NullWritable> {
+
+    @Override
+    public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        return new TextVertexWriterToEachLine() {
+
+            @Override
+            protected Text convertVertexToLine(Vertex<LongWritable, LongWritable, NullWritable, ?> vertex)
+                    throws IOException {
+                return new Text(vertex.getId() + " " + vertex.getValue());
+            }
+
+        };
+    }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
similarity index 64%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
index 3d85f66..38eef3a 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+package edu.uci.ics.pregelix.benchmark.io;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +25,13 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextPRInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,9 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
+            protected LongWritable getId(Text line) throws IOException {
                 items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
@@ -50,14 +50,14 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
                     edge.setEntry(entry);
                     edge.setValue(null);
                     edges.add(edge);
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java
new file mode 100644
index 0000000..b14de6f
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java
@@ -0,0 +1,43 @@
+/*
+ * 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.benchmark.io;
+
+import java.io.IOException;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextPROutputFormat extends TextVertexOutputFormat<LongWritable, DoubleWritable, NullWritable> {
+
+    @Override
+    public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        return new TextVertexWriterToEachLine() {
+
+            @Override
+            protected Text convertVertexToLine(Vertex<LongWritable, DoubleWritable, NullWritable, ?> vertex)
+                    throws IOException {
+                return new Text(vertex.getId() + " " + vertex.getValue());
+            }
+
+        };
+    }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
similarity index 65%
rename from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
rename to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
index 3d85f66..953e93c 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
@@ -12,7 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +26,12 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextSPInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,9 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
+            protected LongWritable getId(Text line) throws IOException {
                 items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
@@ -50,16 +50,16 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, DoubleWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, DoubleWritable>> edges = new ArrayList<Edge<LongWritable, DoubleWritable>>();
+                Map<LongWritable, DoubleWritable> edgeMap = new HashMap<LongWritable, DoubleWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
                     edge.setEntry(entry);
-                    edge.setValue(null);
+                    edge.setValue(new DoubleWritable(1.0));
                     edges.add(edge);
                 }
                 return edges;
@@ -67,4 +67,5 @@
 
         };
     }
+
 }
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java
new file mode 100644
index 0000000..0a70b3c
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java
@@ -0,0 +1,72 @@
+/*
+ * 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.benchmark.io2;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.MapMutableEdge;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextCCInputFormat2 extends TextVertexInputFormat<LongWritable, LongWritable, NullWritable> {
+
+    @Override
+    public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
+        return new TextVertexReaderFromEachLine() {
+            String[] items;
+
+            @Override
+            protected LongWritable getId(Text line) throws IOException {
+                String[] kv = line.toString().split("\t");
+                items = kv[1].split(" ");
+                return new LongWritable(Long.parseLong(kv[0]));
+            }
+
+            @Override
+            protected LongWritable getValue(Text line) throws IOException {
+                return null;
+            }
+
+            @Override
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
+                for (int i = 1; i < items.length; i++) {
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
+                }
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
+                    edge.setEntry(entry);
+                    edge.setValue(null);
+                    edges.add(edge);
+                }
+                return edges;
+            }
+
+        };
+    }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
similarity index 62%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
index 3d85f66..63a4519 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+package edu.uci.ics.pregelix.benchmark.io2;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +25,13 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextPRInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,10 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
-                items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+            protected LongWritable getId(Text line) throws IOException {
+                String[] kv = line.toString().split("\t");
+                items = kv[1].split(" ");
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
@@ -50,14 +51,14 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
                     edge.setEntry(entry);
                     edge.setValue(null);
                     edges.add(edge);
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
similarity index 63%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
index 3d85f66..fdb1061 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
@@ -12,7 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io2;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +26,12 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextSPInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,10 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
-                items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+            protected LongWritable getId(Text line) throws IOException {
+                String[] kv = line.toString().split("\t");
+                items = kv[1].split(" ");
+                return new LongWritable(Long.parseLong(kv[0]));
             }
 
             @Override
@@ -50,16 +51,16 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, DoubleWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, DoubleWritable>> edges = new ArrayList<Edge<LongWritable, DoubleWritable>>();
+                Map<LongWritable, DoubleWritable> edgeMap = new HashMap<LongWritable, DoubleWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
                     edge.setEntry(entry);
-                    edge.setValue(null);
+                    edge.setValue(new DoubleWritable(1.0));
                     edges.add(edge);
                 }
                 return edges;
@@ -67,4 +68,5 @@
 
         };
     }
+
 }
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java
new file mode 100644
index 0000000..3789d6d
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java
@@ -0,0 +1,78 @@
+/*
+ * 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.benchmark.vertex;
+
+import java.io.IOException;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+public class ConnectedComponentsVertex extends Vertex<LongWritable, LongWritable, NullWritable, LongWritable> {
+    /**
+     * Propagates the smallest vertex id to all neighbors. Will always choose to
+     * halt and only reactivate if a smaller id has been sent to it.
+     * 
+     * @param messages
+     *            Iterator of messages from the previous superstep.
+     * @throws IOException
+     */
+    @Override
+    public void compute(Iterable<LongWritable> messages) throws IOException {
+        long currentComponent = getValue().get();
+
+        // First superstep is special, because we can simply look at the neighbors
+        if (getSuperstep() == 0) {
+            for (Edge<LongWritable, NullWritable> edge : getEdges()) {
+                long neighbor = edge.getTargetVertexId().get();
+                if (neighbor < currentComponent) {
+                    currentComponent = neighbor;
+                }
+            }
+            // Only need to send value if it is not the own id
+            if (currentComponent != getValue().get()) {
+                setValue(new LongWritable(currentComponent));
+                for (Edge<LongWritable, NullWritable> edge : getEdges()) {
+                    LongWritable neighbor = edge.getTargetVertexId();
+                    if (neighbor.get() > currentComponent) {
+                        sendMessage(neighbor, getValue());
+                    }
+                }
+            }
+
+            voteToHalt();
+            return;
+        }
+
+        boolean changed = false;
+        // did we get a smaller id ?
+        for (LongWritable message : messages) {
+            long candidateComponent = message.get();
+            if (candidateComponent < currentComponent) {
+                currentComponent = candidateComponent;
+                changed = true;
+            }
+        }
+
+        // propagate new component id to the neighbors
+        if (changed) {
+            setValue(new LongWritable(currentComponent));
+            sendMessageToAllEdges(getValue());
+        }
+        voteToHalt();
+    }
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java
new file mode 100644
index 0000000..86e90dd
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java
@@ -0,0 +1,48 @@
+/*
+ * 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.benchmark.vertex;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.examples.RandomWalkVertex;
+import org.apache.giraph.utils.MathUtils;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+/**
+ * The PageRank algorithm, with uniform transition probabilities on the edges
+ * http://en.wikipedia.org/wiki/PageRank
+ */
+public class PageRankVertex extends RandomWalkVertex<NullWritable> {
+
+    @Override
+    protected double transitionProbability(double stateProbability, Edge<LongWritable, NullWritable> edge) {
+        return stateProbability / getNumEdges();
+    }
+
+    @Override
+    protected double recompute(Iterable<DoubleWritable> partialRanks, double teleportationProbability) {
+
+        // rank contribution from incident neighbors
+        double rankFromNeighbors = MathUtils.sum(partialRanks);
+        // rank contribution from dangling vertices
+        double danglingContribution = getDanglingProbability() / getTotalNumVertices();
+
+        // recompute rank
+        return (1d - teleportationProbability) * (rankFromNeighbors + danglingContribution) + teleportationProbability
+                / getTotalNumVertices();
+    }
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java
new file mode 100644
index 0000000..755a3d0
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.benchmark.vertex;
+
+import java.io.IOException;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Shortest paths algorithm.
+ */
+public class ShortestPathsVertex extends Vertex<LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> {
+    /** Source id. */
+    public static final String SOURCE_ID = "giraph.shortestPathsBenchmark.sourceId";
+    /** Default source id. */
+    public static final long SOURCE_ID_DEFAULT = 1;
+
+    private boolean isSource() {
+        return getId().get() == getConf().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+    }
+
+    @Override
+    public void compute(Iterable<DoubleWritable> messages) throws IOException {
+        if (getSuperstep() == 0) {
+            setValue(new DoubleWritable(Double.MAX_VALUE));
+        }
+
+        double minDist = isSource() ? 0d : Double.MAX_VALUE;
+        for (DoubleWritable message : messages) {
+            minDist = Math.min(minDist, message.get());
+        }
+
+        if (minDist < getValue().get()) {
+            setValue(new DoubleWritable(minDist));
+            for (Edge<LongWritable, DoubleWritable> edge : getEdges()) {
+                double distance = minDist + edge.getValue().get();
+                sendMessage(edge.getTargetVertexId(), new DoubleWritable(distance));
+            }
+        }
+
+        voteToHalt();
+    }
+}
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 a577bdb..8d9eeac 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
@@ -37,7 +37,6 @@
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
@@ -65,6 +64,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
@@ -97,7 +97,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.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ClearStateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -132,7 +131,7 @@
     protected PregelixJob pregelixJob;
     protected IIndexLifecycleManagerProvider lcManagerProvider = IndexLifeCycleManagerProvider.INSTANCE;
     protected IStorageManagerInterface storageManagerInterface = StorageManagerInterface.INSTANCE;
-    protected String jobId = new UUID(System.currentTimeMillis(), System.nanoTime()).toString();
+    protected String jobId = UUID.randomUUID().toString();
     protected int frameSize = ClusterConfig.getFrameSize();
     protected int maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
 
@@ -204,7 +203,6 @@
         return jobId;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     public JobSpecification generateCreatingJob() throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -213,8 +211,7 @@
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);
 
         int[] keyFields = new int[1];
         keyFields[0] = 0;
@@ -257,7 +254,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IConfigurationFactory confFactory = new ConfigurationFactory(conf);
         String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
@@ -272,8 +269,7 @@
         sortFields[0] = 0;
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);;
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameLimit, sortFields,
                 nkmFactory, comparatorFactories, recordDescriptor);
         ClusterConfig.setLocationConstraint(spec, sorter);
@@ -287,7 +283,7 @@
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), vertexClass.getName());
+                conf, vertexIdClass.getName(), vertexClass.getName());
         VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
                 resultFileSplitProvider, preHookFactory, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
@@ -299,12 +295,12 @@
         ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
-                comparatorFactories), sorter, 0, writer, 0);
+                comparatorFactories, nkmFactory), sorter, 0, writer, 0);
         spec.setFrameSize(frameSize);
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
+    @SuppressWarnings({ "rawtypes" })
     public JobSpecification scanIndexPrintGraph(String nodeName, String path) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
@@ -329,11 +325,10 @@
          * construct btree search operator
          */
         IConfigurationFactory confFactory = new ConfigurationFactory(conf);
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
@@ -352,7 +347,7 @@
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), vertexClass.getName());
+                conf, vertexIdClass.getName(), vertexClass.getName());
         VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
                 resultFileSplitProvider, preHookFactory, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
@@ -363,16 +358,17 @@
          */
         int[] sortFields = new int[1];
         sortFields[0] = 0;
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
-                comparatorFactories), scanner, 0, writer, 0);
+                comparatorFactories, nkmFactory), scanner, 0, writer, 0);
         spec.setFrameSize(frameSize);
         return spec;
     }
 
     public JobSpecification scanIndexWriteGraph() throws HyracksException {
-        JobSpecification spec = scanIndexWriteToHDFS(conf);
+        JobSpecification spec = scanIndexWriteToHDFS(conf, false);
         return spec;
     }
 
@@ -388,7 +384,7 @@
             FileSystem dfs = FileSystem.get(tmpJob.getConfiguration());
 
             dfs.delete(new Path(BspUtils.getVertexCheckpointPath(conf, lastSuccessfulIteration)), true);
-            JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration());
+            JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration(), true);
 
             dfs.delete(new Path(BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration)), true);
             JobSpecification[] stateCkpSpecs = generateStateCheckpointing(lastSuccessfulIteration);
@@ -482,7 +478,7 @@
             return new VertexPartitionComputerFactory(confFactory);
         } else {
             return new VertexIdPartitionComputerFactory(new WritableSerializerDeserializerFactory(
-                    BspUtils.getVertexIndexClass(conf)));
+                    BspUtils.getVertexIndexClass(conf)), confFactory);
         }
     }
 
@@ -518,7 +514,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IConfigurationFactory confFactory = new ConfigurationFactory(conf);
         String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
@@ -533,8 +529,7 @@
         sortFields[0] = 0;
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);;
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, sortFields,
                 nkmFactory, comparatorFactories, recordDescriptor);
         ClusterConfig.setLocationConstraint(spec, sorter);
@@ -564,8 +559,9 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    private JobSpecification scanIndexWriteToHDFS(Configuration conf) throws HyracksDataException, HyracksException {
+    @SuppressWarnings({ "rawtypes" })
+    private JobSpecification scanIndexWriteToHDFS(Configuration conf, boolean ckpointing) throws HyracksDataException,
+            HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
         JobSpecification spec = new JobSpecification();
@@ -589,11 +585,10 @@
          * construct btree search operator
          */
         IConfigurationFactory confFactory = new ConfigurationFactory(conf);
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -605,20 +600,37 @@
                 null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
+        ExternalSortOperatorDescriptor sort = null;
+        if (!ckpointing) {
+            int[] keyFields = new int[] { 0 };
+            INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getFinalNormalizedKeyComputerFactory(conf);
+            IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+            sortCmpFactories[0] = JobGenUtil.getFinalBinaryComparatorFactory(vertexIdClass);
+            sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields, nkmFactory, sortCmpFactories,
+                    recordDescriptor);
+            ClusterConfig.setLocationConstraint(spec, scanner);
+        }
+
         /**
          * construct write file operator
          */
+        IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), vertexClass.getName());
+                conf, vertexIdClass.getName(), vertexClass.getName());
         VertexFileWriteOperatorDescriptor writer = new VertexFileWriteOperatorDescriptor(spec, confFactory,
-                inputRdFactory);
+                inputRdFactory, preHookFactory);
         ClusterConfig.setLocationConstraint(spec, writer);
 
         /**
          * connect operator descriptors
          */
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+        if (!ckpointing) {
+            spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sort, 0);
+            spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, writer, 0);
+        } else {
+            spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+        }
         spec.setFrameSize(frameSize);
         return spec;
     }
@@ -645,7 +657,7 @@
         /**
          * source aggregate
          */
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
 
         /**
@@ -658,7 +670,7 @@
          * construct the materializing write operator
          */
         MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
-                false);
+                false, jobId, lastSuccessfulIteration + 1);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         String checkpointPath = BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration);;
@@ -669,7 +681,7 @@
         tmpJob.setOutputValueClass(MsgList.class);
 
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), MsgList.class.getName());
+                conf, vertexIdClass.getName(), MsgList.class.getName());
         HDFSFileWriteOperatorDescriptor hdfsWriter = new HDFSFileWriteOperatorDescriptor(spec, tmpJob, inputRdFactory);
         ClusterConfig.setLocationConstraint(spec, hdfsWriter);
 
@@ -709,7 +721,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), MsgList.class.getName());
         String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
         HDFSReadOperatorDescriptor scanner = new HDFSReadOperatorDescriptor(spec, recordDescriptor, tmpJob, splits,
@@ -720,17 +732,16 @@
         int[] keyFields = new int[] { 0 };
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastCheckpointedIteration,
-                WritableComparator.get(vertexIdClass).getClass());
+        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastCheckpointedIteration, vertexIdClass);
         ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, recordDescriptor);
+                nkmFactory, sortCmpFactories, recordDescriptor, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, sort);
 
         /**
          * construct the materializing write operator
          */
         MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec,
-                recordDescriptor);
+                recordDescriptor, jobId, lastCheckpointedIteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         /** construct runtime hook */
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 9389f62..9fefbea 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
@@ -50,6 +50,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -62,7 +63,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.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -97,7 +97,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
@@ -126,30 +125,30 @@
         /**
          * construct btree search and function call update operator
          */
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
 
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), vertexClass.getName());
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+                conf, vertexIdClass.getName(), vertexClass.getName());
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
@@ -168,8 +167,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -195,7 +194,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -219,7 +218,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         /**
@@ -282,8 +282,8 @@
 
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 5, btreeBulkLoad, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
@@ -300,7 +300,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -313,16 +312,15 @@
          * source aggregate
          */
         int[] keyFields = new int[] { 0 };
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);;
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -342,7 +340,7 @@
          * construct the materializing write operator
          */
         MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
-                true);
+                true, jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
@@ -363,13 +361,14 @@
          * construct index-join-function-update operator
          */
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
+                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
@@ -401,7 +400,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -425,7 +424,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         /** construct runtime hook */
@@ -447,8 +447,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -503,8 +503,8 @@
 
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 5, btreeBulkLoad, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
@@ -587,7 +587,7 @@
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), MsgList.class.getName());
         String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
         HDFSReadOperatorDescriptor scanner = new HDFSReadOperatorDescriptor(spec, recordDescriptor, tmpJob, splits,
@@ -598,10 +598,9 @@
         int[] keyFields = new int[] { 0 };
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastSuccessfulIteration,
-                WritableComparator.get(vertexIdClass).getClass());
+        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastSuccessfulIteration, vertexIdClass);
         ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, recordDescriptor);
+                nkmFactory, sortCmpFactories, recordDescriptor, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, sort);
 
         /**
@@ -633,7 +632,7 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
+    @SuppressWarnings({ "rawtypes" })
     private JobSpecification generateSecondaryBTreeCheckpoint(int lastSuccessfulIteration, PregelixJob job)
             throws HyracksException {
         job.setOutputFormatClass(SequenceFileOutputFormat.class);
@@ -665,11 +664,10 @@
         /**
          * construct btree search operator
          */
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), msgListClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);;
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
@@ -685,7 +683,7 @@
          * construct write file operator
          */
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), MsgList.class.getName());
+                conf, vertexIdClass.getName(), MsgList.class.getName());
         HDFSFileWriteOperatorDescriptor writer = new HDFSFileWriteOperatorDescriptor(spec, job, inputRdFactory);
         ClusterConfig.setLocationConstraint(spec, writer);
 
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 287b797..ee576b1 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
@@ -17,7 +17,6 @@
 import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -32,6 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -42,7 +42,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.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -74,7 +73,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -98,29 +96,29 @@
         /**
          * construct btree search function update operator
          */
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
 
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), vertexClass.getName());
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+                conf, vertexIdClass.getName(), vertexClass.getName());
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
@@ -135,10 +133,9 @@
         int[] keyFields = new int[] { 0 };
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
+        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -153,7 +150,7 @@
         /**
          * construct global group-by operator
          */
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
         IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
                 conf, true, true);
@@ -164,7 +161,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
@@ -186,8 +184,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -241,8 +239,8 @@
          * connect the group-by operator
          */
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink2, 0);
@@ -258,7 +256,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -271,16 +268,15 @@
          * source aggregate
          */
         int[] keyFields = new int[] { 0 };
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -300,7 +296,7 @@
          * construct the materializing write operator
          */
         MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
-                true);
+                true, jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
@@ -314,13 +310,14 @@
         nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
         nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
 
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
+                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
@@ -335,10 +332,9 @@
          */
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
+        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -362,7 +358,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         /** construct runtime hook */
@@ -384,8 +381,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -440,8 +437,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
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 3b3c9e7..32271c8 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
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -41,7 +42,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.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -73,7 +73,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -97,11 +96,10 @@
         /**
          * construct btree search operator
          */
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -111,18 +109,19 @@
         /**
          * construct compute operator
          */
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), vertexClass.getName());
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+                conf, vertexIdClass.getName(), vertexClass.getName());
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
@@ -140,13 +139,13 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
          * construct global group-by operator
          */
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
         IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
                 conf, true, false);
@@ -157,7 +156,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
@@ -178,8 +178,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -247,7 +247,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -260,16 +259,15 @@
          * source aggregate
          */
         int[] keyFields = new int[] { 0 };
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -289,7 +287,7 @@
          * construct the materializing write operator
          */
         MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
-                true);
+                true, jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
@@ -303,13 +301,14 @@
         nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
         nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
 
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
+                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
@@ -327,7 +326,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
@@ -342,7 +341,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         /** construct runtime hook */
@@ -364,8 +364,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
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 e334095..3aa36cd 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
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -41,7 +42,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.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -73,7 +73,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -97,29 +96,29 @@
         /**
          * construct btree search function update operator
          */
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
 
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), vertexClass.getName());
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+                conf, vertexIdClass.getName(), vertexClass.getName());
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
@@ -137,7 +136,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -153,13 +152,13 @@
          * construct global sort operator
          */
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
          * construct global group-by operator
          */
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
         IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
                 conf, true, true);
@@ -170,7 +169,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
@@ -192,8 +192,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -261,7 +261,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -274,16 +273,15 @@
          * source aggregate
          */
         int[] keyFields = new int[] { 0 };
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
+        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName());
+        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
 
         /**
          * construct empty tuple operator
@@ -303,7 +301,7 @@
          * construct the materializing write operator
          */
         MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
-                true);
+                true, jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materializeRead);
 
         /**
@@ -317,13 +315,14 @@
         nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
         nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
 
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils
-                .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName());
+        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                VLongWritable.class.getName());
+        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
+                partialAggregateValueClass.getName());
         IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
+                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
@@ -341,7 +340,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -357,7 +356,7 @@
          * construct global sort operator
          */
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
@@ -372,7 +371,8 @@
         /**
          * construct the materializing write operator
          */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal);
+        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
+                jobId, iteration);
         ClusterConfig.setLocationConstraint(spec, materialize);
 
         /** construct runtime hook */
@@ -394,8 +394,8 @@
         /**
          * final aggregate write operator
          */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils
-                .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName());
+        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+                conf, partialAggregateValueClass.getName());
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
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 c4f54fe..9f2a66d 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
@@ -16,15 +16,18 @@
 package edu.uci.ics.pregelix.core.jobgen;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableComparator;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
 import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawNormalizedKeyComputerFactory;
 import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNormalizedKeyComputerFactory;
 
-@SuppressWarnings({ "rawtypes", "unchecked" })
+@SuppressWarnings({ "rawtypes" })
 public class JobGenUtil {
 
     /**
@@ -36,11 +39,7 @@
      * @return
      */
     public static INormalizedKeyComputerFactory getINormalizedKeyComputerFactory(Configuration conf) {
-        Class<? extends NormalizedKeyComputer> clazz = BspUtils.getNormalizedKeyComputerClass(conf);
-        if (clazz.equals(NormalizedKeyComputer.class)) {
-            return null;
-        }
-        return new VertexIdNormalizedKeyComputerFactory(clazz);
+        return RawNormalizedKeyComputerFactory.INSTANCE;
     }
 
     /**
@@ -52,7 +51,34 @@
      * @return
      */
     public static IBinaryComparatorFactory getIBinaryComparatorFactory(int iteration, Class keyClass) {
-        return new WritableComparingBinaryComparatorFactory(keyClass);
+        return RawBinaryComparatorFactory.INSTANCE;
+    }
+
+    /**
+     * get normalized key factory for the final output job
+     * 
+     * @param iteration
+     * @param keyClass
+     * @return
+     */
+    public static INormalizedKeyComputerFactory getFinalNormalizedKeyComputerFactory(Configuration conf) {
+        Class<? extends NormalizedKeyComputer> clazz = BspUtils.getNormalizedKeyComputerClass(conf);
+        if (clazz.equals(NormalizedKeyComputer.class)) {
+            return null;
+        }
+        return new VertexIdNormalizedKeyComputerFactory(clazz);
+    }
+
+    /**
+     * get comparator for the final output job
+     * 
+     * @param iteration
+     * @param keyClass
+     * @return
+     */
+    @SuppressWarnings("unchecked")
+    public static IBinaryComparatorFactory getFinalBinaryComparatorFactory(Class vertexIdClass) {
+        return new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass).getClass());
     }
 
     /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java
new file mode 100644
index 0000000..c0173ed
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java
@@ -0,0 +1,46 @@
+/*
+ * 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.core.runtime.touchpoint;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static IBinaryComparatorFactory INSTANCE = new RawBinaryComparatorFactory();
+
+    private RawBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int commonLength = Math.min(l1, l2);
+                for (int i = 0; i < commonLength; i++) {
+                    if (b1[s1 + i] != b2[s2 + i]) {
+                        return (b1[s1 + i] & 0xff) - (b2[s2 + i] & 0xff);
+                    }
+                }
+                int difference = l1 - l2;
+                return difference == 0 ? 0 : (difference > 0 ? 1 : -1);
+            }
+
+        };
+    }
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..6aa4af9
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.core.runtime.touchpoint;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class RawNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+    private static final long serialVersionUID = 1L;
+    public static final INormalizedKeyComputerFactory INSTANCE = new RawNormalizedKeyComputerFactory();
+
+    private RawNormalizedKeyComputerFactory() {
+
+    }
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                int nk = 0;
+                for (int i = 0; i < 4; i++) {
+                    nk <<= 8;
+                    if (i < length) {
+                        nk += (bytes[start + i] & 0xff);
+                    }
+                }
+                return nk;
+            }
+
+        };
+    }
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java
index a67c259..68e3ba7 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java
@@ -14,25 +14,32 @@
  */
 package edu.uci.ics.pregelix.core.runtime.touchpoint;
 
+import org.apache.hadoop.conf.Configuration;
+
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 
 public class WritableRecordDescriptorFactory implements IRecordDescriptorFactory {
     private static final long serialVersionUID = 1L;
     private String[] fieldClasses;
+    private IConfigurationFactory confFactory;
 
-    public WritableRecordDescriptorFactory(String... fieldClasses) {
+    public WritableRecordDescriptorFactory(Configuration conf, String... fieldClasses) {
         this.fieldClasses = fieldClasses;
+        this.confFactory = new ConfigurationFactory(conf);
     }
 
     @Override
     public RecordDescriptor createRecordDescriptor(IHyracksTaskContext ctx) throws HyracksDataException {
         try {
-            return DataflowUtils.getRecordDescriptorFromWritableClasses(ctx, fieldClasses);
+            Configuration conf = confFactory.createConfiguration(ctx);
+            return DataflowUtils.getRecordDescriptorFromWritableClasses(ctx, conf, fieldClasses);
         } catch (HyracksException e) {
             throw new HyracksDataException(e);
         }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java
index 3e01109..3a2241b 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java
@@ -38,14 +38,14 @@
     }
 
     @SuppressWarnings("unchecked")
-    public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(String className1, String className2)
-            throws HyracksException {
+    public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(Configuration conf, String className1,
+            String className2) throws HyracksException {
         RecordDescriptor recordDescriptor = null;
         try {
             ClassLoader loader = DataflowUtils.class.getClassLoader();
             recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
                     (Class<? extends Writable>) loader.loadClass(className1),
-                    (Class<? extends Writable>) loader.loadClass(className2));
+                    (Class<? extends Writable>) loader.loadClass(className2), conf);
         } catch (ClassNotFoundException cnfe) {
             throw new HyracksException(cnfe);
         }
@@ -53,15 +53,16 @@
     }
 
     @SuppressWarnings({ "unchecked", "rawtypes" })
-    public static RecordDescriptor getRecordDescriptorFromWritableClasses(String... classNames) throws HyracksException {
+    public static RecordDescriptor getRecordDescriptorFromWritableClasses(Configuration conf, String... classNames)
+            throws HyracksException {
         RecordDescriptor recordDescriptor = null;
         ISerializerDeserializer[] serdes = new ISerializerDeserializer[classNames.length];
         ClassLoader loader = DataflowUtils.class.getClassLoader();
         try {
             int i = 0;
             for (String className : classNames)
-                serdes[i++] = DatatypeHelper.createSerializerDeserializer((Class<? extends Writable>) loader
-                        .loadClass(className));
+                serdes[i++] = DatatypeHelper.createSerializerDeserializer(
+                        (Class<? extends Writable>) loader.loadClass(className), conf);
         } catch (ClassNotFoundException cnfe) {
             throw new HyracksException(cnfe);
         }
@@ -69,9 +70,9 @@
         return recordDescriptor;
     }
 
-    public static IRecordDescriptorFactory getWritableRecordDescriptorFactoryFromWritableClasses(String... classNames)
-            throws HyracksException {
-        IRecordDescriptorFactory rdFactory = new WritableRecordDescriptorFactory(classNames);
+    public static IRecordDescriptorFactory getWritableRecordDescriptorFactoryFromWritableClasses(Configuration conf,
+            String... classNames) throws HyracksException {
+        IRecordDescriptorFactory rdFactory = new WritableRecordDescriptorFactory(conf, classNames);
         return rdFactory;
     }
 
@@ -85,13 +86,13 @@
     }
 
     @SuppressWarnings("unchecked")
-    public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(IHyracksTaskContext ctx, String className1,
-            String className2) throws HyracksException {
+    public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(IHyracksTaskContext ctx, Configuration conf,
+            String className1, String className2) throws HyracksException {
         RecordDescriptor recordDescriptor = null;
         try {
             recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class<? extends Writable>) ctx
                     .getJobletContext().loadClass(className1), (Class<? extends Writable>) ctx.getJobletContext()
-                    .loadClass(className2));
+                    .loadClass(className2), conf);
         } catch (Exception cnfe) {
             throw new HyracksException(cnfe);
         }
@@ -99,15 +100,17 @@
     }
 
     @SuppressWarnings({ "unchecked", "rawtypes" })
-    public static RecordDescriptor getRecordDescriptorFromWritableClasses(IHyracksTaskContext ctx, String... classNames)
-            throws HyracksException {
+    public static RecordDescriptor getRecordDescriptorFromWritableClasses(IHyracksTaskContext ctx, Configuration conf,
+            String... classNames) throws HyracksException {
         RecordDescriptor recordDescriptor = null;
         ISerializerDeserializer[] serdes = new ISerializerDeserializer[classNames.length];
         try {
             int i = 0;
-            for (String className : classNames)
-                serdes[i++] = DatatypeHelper.createSerializerDeserializer((Class<? extends Writable>) ctx
-                        .getJobletContext().loadClass(className));
+            for (String className : classNames) {
+                Class<? extends Writable> c = (Class<? extends Writable>) ctx.getJobletContext().loadClass(className);
+                serdes[i++] = DatatypeHelper.createSerializerDeserializer(c, conf);
+                //System.out.println("thread " + Thread.currentThread().getId() + " after creating serde " + c.getClassLoader());
+            }
         } catch (Exception cnfe) {
             throw new HyracksException(cnfe);
         }
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 aabd4ba..70de9ed 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
@@ -62,7 +62,7 @@
         ccConfig.jobHistorySize = 1;
         ccConfig.profileDumpPeriod = -1;
         ccConfig.heartbeatPeriod = 50;
-        ccConfig.maxHeartbeatLapsePeriods = 15;
+        ccConfig.maxHeartbeatLapsePeriods = 10;
 
         // cluster controller
         cc = new ClusterControllerService(ccConfig);
diff --git a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
index 4c7f91d..f599996 100644
--- a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
+++ b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 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.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -34,6 +35,7 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -89,6 +91,7 @@
             UTF8StringPointable.FACTORY);
     private IBinaryComparatorFactory stringComparatorFactory = new PointableBinaryComparatorFactory(
             UTF8StringPointable.FACTORY);
+    private INormalizedKeyComputerFactory nmkFactory = new UTF8StringNormalizedKeyComputerFactory();
 
     private void cleanupStores() throws IOException {
         FileUtils.forceMkdir(new File("teststore"));
@@ -211,7 +214,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, sorter, 0);
         IConnectorDescriptor joinWriterConn = new MToNPartitioningMergingConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories);
+                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory);
         spec.connect(joinWriterConn, sorter, 0, writer, 0);
 
         spec.addRoot(writer);
@@ -284,8 +288,8 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                sortFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
-                sorter, 0, writer, 0);
+                sortFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory), sorter, 0, writer, 0);
 
         spec.addRoot(writer);
         runTest(spec);
@@ -368,11 +372,11 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
-                sorter, 0, join, 0);
+                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory), sorter, 0, join, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
-                join, 0, writer, 0);
+                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory), join, 0, writer, 0);
 
         spec.addRoot(writer);
         runTest(spec);
@@ -477,7 +481,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), project, 0, sorter, 0);
         IConnectorDescriptor joinWriterConn = new MToNPartitioningMergingConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 9 },
-                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories);
+                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory);
         spec.connect(joinWriterConn, sorter, 0, writer, 0);
 
         spec.addRoot(writer);
@@ -573,7 +578,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
                 keyFields, new IBinaryHashFunctionFactory[] { new PointableBinaryHashFunctionFactory(
-                        UTF8StringPointable.FACTORY) }), sortFields, comparatorFactories), sorter, 0, join, 0);
+                        UTF8StringPointable.FACTORY) }), sortFields, comparatorFactories, nmkFactory), sorter, 0, join,
+                0);
 
         IBinaryComparatorFactory[] mergeComparatorFactories = new IBinaryComparatorFactory[2];
         mergeComparatorFactories[0] = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY);
@@ -581,7 +587,8 @@
         int[] mergeFields = new int[] { 9, 0 };
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
                 new int[] { 9 }, new IBinaryHashFunctionFactory[] { new PointableBinaryHashFunctionFactory(
-                        UTF8StringPointable.FACTORY) }), mergeFields, comparatorFactories), join, 0, writer, 0);
+                        UTF8StringPointable.FACTORY) }), mergeFields, comparatorFactories, nmkFactory), join, 0,
+                writer, 0);
 
         spec.addRoot(writer);
         runTest(spec);
diff --git a/pregelix/pregelix-dataflow-std-base/pom.xml b/pregelix/pregelix-dataflow-std-base/pom.xml
index d4c0ee6..77d75bf 100644
--- a/pregelix/pregelix-dataflow-std-base/pom.xml
+++ b/pregelix/pregelix-dataflow-std-base/pom.xml
@@ -1,28 +1,24 @@
-<!--
- ! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<!-- ! 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. ! -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<artifactId>pregelix-dataflow-std-base</artifactId>
 	<packaging>jar</packaging>
 	<name>pregelix-dataflow-std-base</name>
 
 	<parent>
-    		<groupId>edu.uci.ics.hyracks</groupId>
-    		<artifactId>pregelix</artifactId>
-    		<version>0.2.10-SNAPSHOT</version>
-  	</parent>
+		<groupId>edu.uci.ics.hyracks</groupId>
+		<artifactId>pregelix</artifactId>
+		<version>0.2.10-SNAPSHOT</version>
+	</parent>
 
 
 	<properties>
@@ -58,7 +54,7 @@
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-clean-plugin</artifactId>
-                <version>2.4.1</version>
+				<version>2.4.1</version>
 				<configuration>
 					<filesets>
 						<fileset>
@@ -94,6 +90,20 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-hdfs-core</artifactId>
+			<version>0.2.10-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-common</artifactId>
+			<version>0.2.10-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-api</artifactId>
 			<version>0.2.10-SNAPSHOT</version>
 			<type>jar</type>
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java
index 1fdd0b6..894cba5 100644
--- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java
@@ -16,10 +16,12 @@
 
 import java.io.Serializable;
 
+import org.apache.hadoop.conf.Configuration;
+
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 
 public interface ISerializerDeserializerFactory<T> extends Serializable {
 
-	public ISerializerDeserializer<T> getSerializerDeserializer();
+    public ISerializerDeserializer<T> getSerializerDeserializer(Configuration conf);
 
 }
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
index b8ba7bd..081b3bc 100644
--- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java
@@ -18,17 +18,18 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 
 public interface IUpdateFunction extends IFunction {
 
-	/**
-	 * update the tuple pointed by tupleRef called after process,
-	 * one-input-tuple-at-a-time
-	 * 
-	 * @param tupleRef
-	 * @throws HyracksDataException
-	 */
-	public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb)
-			throws HyracksDataException;
+    /**
+     * update the tuple pointed by tupleRef called after process,
+     * one-input-tuple-at-a-time
+     * 
+     * @param tupleRef
+     * @throws HyracksDataException
+     */
+    public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor)
+            throws HyracksDataException;
 
 }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
index b22e468..c9bda52 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
@@ -34,6 +34,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -43,6 +44,7 @@
 import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
 import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.util.StorageType;
 import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
@@ -68,6 +70,7 @@
     private ArrayTupleBuilder cloneUpdateTb;
     private final UpdateBuffer updateBuffer;
     private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference();
+    private final StorageType storageType;
 
     public IndexNestedLoopJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -77,6 +80,11 @@
             throws HyracksDataException {
         treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
+        if (treeIndexOpHelper instanceof TreeIndexDataflowHelper) {
+            storageType = StorageType.TreeIndex;
+        } else {
+            storageType = StorageType.LSMIndex;
+        }
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -96,7 +104,7 @@
     }
 
     protected void setCursor() {
-        cursor = indexAccessor.createSearchCursor();
+        cursor = indexAccessor.createSearchCursor(true);
     }
 
     @Override
@@ -170,13 +178,13 @@
             /**
              * call the update function
              */
-            functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb);
+            functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb, cursor);
 
             /**
              * doing copy update
              */
             CopyUpdateUtil.copyUpdate(tempTupleReference, indexEntryTuple, updateBuffer, cloneUpdateTb, indexAccessor,
-                    cursor, rangePred);
+                    cursor, rangePred, false, storageType);
         }
     }
 
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
index 2ddca90..b5b21b6 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java
@@ -82,7 +82,7 @@
     }
 
     protected void setCursor() {
-        cursor = indexAccessor.createSearchCursor();
+        cursor = indexAccessor.createSearchCursor(false);
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
index 0ecfd03..d77107e 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
@@ -36,6 +36,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -45,6 +46,7 @@
 import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
 import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.util.StorageType;
 import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable extends
@@ -79,6 +81,7 @@
     private ArrayTupleBuilder cloneUpdateTb;
     private final UpdateBuffer updateBuffer;
     private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference();
+    private final StorageType storageType;
 
     public IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -88,6 +91,11 @@
         inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
         treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
+        if (treeIndexOpHelper instanceof TreeIndexDataflowHelper) {
+            storageType = StorageType.TreeIndex;
+        } else {
+            storageType = StorageType.LSMIndex;
+        }
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
 
         if (lowKeyFields != null && lowKeyFields.length > 0) {
@@ -107,7 +115,7 @@
     }
 
     protected void setCursor() {
-        cursor = indexAccessor.createSearchCursor();
+        cursor = indexAccessor.createSearchCursor(true);
     }
 
     @Override
@@ -287,13 +295,13 @@
         /**
          * function call
          */
-        functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb);
+        functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb, cursor);
 
         /**
          * doing clone update
          */
         CopyUpdateUtil.copyUpdate(tempTupleReference, indexEntryTuple, updateBuffer, cloneUpdateTb, indexAccessor,
-                cursor, rangePred);
+                cursor, rangePred, true, storageType);
     }
 
     /** write result for outer case */
@@ -301,11 +309,11 @@
         /**
          * function call
          */
-        functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb);
+        functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb, cursor);
 
         //doing clone update
         CopyUpdateUtil.copyUpdate(tempTupleReference, frameTuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor,
-                rangePred);
+                rangePred, true, storageType);
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
index e64e9cc..6539828 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
@@ -89,7 +89,7 @@
     }
 
     protected void setCursor() {
-        cursor = indexAccessor.createSearchCursor();
+        cursor = indexAccessor.createSearchCursor(false);
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
index a9c787f..6549782 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
@@ -34,6 +34,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -43,6 +44,7 @@
 import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
 import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.util.StorageType;
 import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
@@ -71,6 +73,7 @@
     private ArrayTupleBuilder cloneUpdateTb;
     private UpdateBuffer updateBuffer;
     private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference();
+    private final StorageType storageType;
 
     public IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -79,6 +82,11 @@
             IRecordDescriptorFactory inputRdFactory, int outputArity) throws HyracksDataException {
         treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
+        if (treeIndexOpHelper instanceof TreeIndexDataflowHelper) {
+            storageType = StorageType.TreeIndex;
+        } else {
+            storageType = StorageType.LSMIndex;
+        }
         this.isForward = isForward;
         this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
 
@@ -98,7 +106,7 @@
     }
 
     protected void setCursor() {
-        cursor = indexAccessor.createSearchCursor();
+        cursor = indexAccessor.createSearchCursor(true);
     }
 
     @Override
@@ -238,21 +246,21 @@
 
     /** write the right result */
     private void writeRightResults(ITupleReference frameTuple) throws Exception {
-        functionProxy.functionCall(frameTuple, cloneUpdateTb);
+        functionProxy.functionCall(frameTuple, cloneUpdateTb, cursor);
 
         //doing clone update
         CopyUpdateUtil.copyUpdate(tempTupleReference, frameTuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor,
-                rangePred);
+                rangePred, true, storageType);
     }
 
     /** write the left result */
     private void writeLeftResults(IFrameTupleAccessor leftAccessor, int tIndex, ITupleReference frameTuple)
             throws Exception {
-        functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb);
+        functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb, cursor);
 
         //doing clone update
         CopyUpdateUtil.copyUpdate(tempTupleReference, frameTuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor,
-                rangePred);
+                rangePred, true, storageType);
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
index 86a211f..1c9fce6 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
@@ -82,7 +82,7 @@
     }
 
     protected void setCursor() {
-        cursor = indexAccessor.createSearchCursor();
+        cursor = indexAccessor.createSearchCursor(false);
     }
 
     @Override
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
index de87909..0561f8b 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
@@ -35,6 +35,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -44,6 +45,7 @@
 import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil;
 import edu.uci.ics.pregelix.dataflow.util.FunctionProxy;
 import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.util.StorageType;
 import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer;
 
 public class TreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
@@ -77,6 +79,7 @@
     private ArrayTupleBuilder cloneUpdateTb;
     private final UpdateBuffer updateBuffer;
     private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference();
+    private final StorageType storageType;
 
     public TreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward,
@@ -86,6 +89,11 @@
             throws HyracksDataException {
         treeIndexHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
                 opDesc, ctx, partition);
+        if (treeIndexHelper instanceof TreeIndexDataflowHelper) {
+            storageType = StorageType.TreeIndex;
+        } else {
+            storageType = StorageType.LSMIndex;
+        }
         this.isForward = isForward;
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
@@ -164,18 +172,18 @@
     }
 
     protected void setCursor() {
-        cursor = indexAccessor.createSearchCursor();
+        cursor = indexAccessor.createSearchCursor(true);
     }
 
     protected void writeSearchResults() throws Exception {
         while (cursor.hasNext()) {
             cursor.next();
             ITupleReference tuple = cursor.getTuple();
-            functionProxy.functionCall(tuple, cloneUpdateTb);
+            functionProxy.functionCall(tuple, cloneUpdateTb, cursor);
 
             //doing clone update
             CopyUpdateUtil.copyUpdate(tempTupleReference, tuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor,
-                    rangePred);
+                    rangePred, true, storageType);
         }
     }
 
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 392f728..0ff3f04 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
@@ -27,19 +27,9 @@
 
     public static void copyUpdate(SearchKeyTupleReference tempTupleReference, ITupleReference frameTuple,
             UpdateBuffer updateBuffer, ArrayTupleBuilder cloneUpdateTb, IIndexAccessor indexAccessor,
-            IIndexCursor cursor, RangePredicate rangePred) throws HyracksDataException, IndexException {
+            IIndexCursor cursor, RangePredicate rangePred, boolean scan, StorageType type) 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 the vertex size is not larger than the original size, 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));
@@ -51,11 +41,18 @@
                 if (!updateBuffer.appendTuple(cloneUpdateTb)) {
                     throw new HyracksDataException("cannot append tuple builder!");
                 }
-                //search again and recover the cursor
+                //search again and recover the cursor to the exact point as the one before it is closed
                 cursor.reset();
-                rangePred.setLowKey(tempTupleReference, false);
-                rangePred.setHighKey(null, true);
+                rangePred.setLowKey(tempTupleReference, true);
+                if (scan) {
+                    rangePred.setHighKey(null, true);
+                } else {
+                    rangePred.setHighKey(tempTupleReference, true);
+                }
                 indexAccessor.search(cursor, rangePred);
+                if (cursor.hasNext()) {
+                    cursor.next();
+                }
             }
             cloneUpdateTb.reset();
         }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
index a1e5b86..5579a77 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction;
@@ -56,10 +57,10 @@
      * @throws HyracksDataException
      */
     public void functionOpen() throws HyracksDataException {
+        ctxCL = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(ctx.getJobletContext().getClassLoader());
         inputRd = inputRdFactory.createRecordDescriptor(ctx);
         tupleDe = new TupleDeserializer(inputRd);
-        ctxCL = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
         for (IFrameWriter writer : writers) {
             writer.open();
         }
@@ -80,10 +81,10 @@
      * @throws HyracksDataException
      */
     public void functionCall(IFrameTupleAccessor leftAccessor, int leftTupleIndex, ITupleReference right,
-            ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+            ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) throws HyracksDataException {
         Object[] tuple = tupleDe.deserializeRecord(leftAccessor, leftTupleIndex, right);
         function.process(tuple);
-        function.update(right, cloneUpdateTb);
+        function.update(right, cloneUpdateTb, cursor);
     }
 
     /**
@@ -92,10 +93,11 @@
      * @param updateRef
      * @throws HyracksDataException
      */
-    public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+    public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor)
+            throws HyracksDataException {
         Object[] tuple = tupleDe.deserializeRecord(updateRef);
         function.process(tuple);
-        function.update(updateRef, cloneUpdateTb);
+        function.update(updateRef, cloneUpdateTb, cursor);
     }
 
     /**
@@ -107,11 +109,11 @@
      *            update pointer
      * @throws HyracksDataException
      */
-    public void functionCall(ArrayTupleBuilder tb, ITupleReference inPlaceUpdateRef, ArrayTupleBuilder cloneUpdateTb)
-            throws HyracksDataException {
+    public void functionCall(ArrayTupleBuilder tb, ITupleReference inPlaceUpdateRef, ArrayTupleBuilder cloneUpdateTb,
+            IIndexCursor cursor) throws HyracksDataException {
         Object[] tuple = tupleDe.deserializeRecord(tb, inPlaceUpdateRef);
         function.process(tuple);
-        function.update(inPlaceUpdateRef, cloneUpdateTb);
+        function.update(inPlaceUpdateRef, cloneUpdateTb, cursor);
     }
 
     /**
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
new file mode 100644
index 0000000..fb2d1eb
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
@@ -0,0 +1,21 @@
+/*
+ * 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.dataflow.util;
+
+public enum StorageType {
+    TreeIndex,
+    LSMIndex
+}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
index b44b643..a5d2ab7 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
@@ -31,11 +31,15 @@
 public class MaterializingReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private static final long serialVersionUID = 1L;
     private final boolean removeIterationState;
+    private final String jobId;
+    private final int iteration;
 
     public MaterializingReadOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor,
-            boolean removeIterationState) {
+            boolean removeIterationState, String jobId, int iteration) {
         super(spec, 1, 1);
         this.removeIterationState = removeIterationState;
+        this.jobId = jobId;
+        this.iteration = iteration - 1;
         recordDescriptors[0] = recordDescriptor;
     }
 
@@ -55,8 +59,8 @@
             @Override
             public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
                 if (!complete) {
-                    MaterializerTaskState state = (MaterializerTaskState) IterationUtils.getIterationState(ctx,
-                            partition);
+                    MaterializerTaskState state = (MaterializerTaskState) IterationUtils.getIterationState(ctx, jobId,
+                            partition, iteration);
                     RunFileReader in = state.getRunFileWriter().createReader();
                     writer.open();
                     try {
@@ -85,7 +89,7 @@
                  * remove last iteration's state
                  */
                 if (removeIterationState) {
-                    IterationUtils.removeIterationState(ctx, partition);
+                    IterationUtils.removeIterationState(ctx, jobId, partition, iteration);
                 }
                 writer.close();
                 complete = true;
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java
index 00dcbd1..921dc40 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java
@@ -38,9 +38,14 @@
 public class MaterializingWriteOperatorDescriptor extends AbstractOperatorDescriptor {
     private static final long serialVersionUID = 1L;
     private final static int MATERIALIZER_ACTIVITY_ID = 0;
+    private final String jobId;
+    private final int iteration;
 
-    public MaterializingWriteOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor) {
+    public MaterializingWriteOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor, String jobId,
+            int iteration) {
         super(spec, 1, 1);
+        this.jobId = jobId;
+        this.iteration = iteration;
         recordDescriptors[0] = recordDescriptor;
     }
 
@@ -69,13 +74,12 @@
                 @Override
                 public void open() throws HyracksDataException {
                     /** remove last iteration's state */
-                    IterationUtils.removeIterationState(ctx, partition);
+                    IterationUtils.removeIterationState(ctx, jobId, partition, iteration);
                     state = new MaterializerTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
                             partition));
                     INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
                     RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
-                    FileReference file = context.createManagedWorkspaceFile(MaterializingWriteOperatorDescriptor.class
-                            .getSimpleName());
+                    FileReference file = context.createManagedWorkspaceFile(jobId);
                     state.setRunFileWriter(new RunFileWriter(file, ctx.getIOManager()));
                     state.getRunFileWriter().open();
                     writer.open();
@@ -92,7 +96,7 @@
                     /**
                      * set iteration state
                      */
-                    IterationUtils.setIterationState(ctx, partition, state);
+                    IterationUtils.setIterationState(ctx, jobId, partition, iteration, state);
                     writer.close();
                 }
 
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
index b34879e..e16ba48 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java
@@ -16,6 +16,8 @@
 
 import java.io.DataOutput;
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -122,7 +124,7 @@
             @SuppressWarnings("unchecked")
             private void loadVertices(final IHyracksTaskContext ctx, Configuration conf, int splitId)
                     throws IOException, ClassNotFoundException, InterruptedException, InstantiationException,
-                    IllegalAccessException {
+                    IllegalAccessException, NoSuchFieldException, InvocationTargetException {
                 ByteBuffer frame = ctx.allocateFrame();
                 FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
                 appender.reset(frame, true);
@@ -141,7 +143,11 @@
                 /**
                  * set context
                  */
-                Vertex.setContext(mapperContext);
+                ClassLoader cl = ctx.getJobletContext().getClassLoader();
+                Class<?> vClass = (Class<?>) cl.loadClass("edu.uci.ics.pregelix.api.graph.Vertex");
+                Field contextField = vClass.getDeclaredField("context");
+                contextField.setAccessible(true);
+                contextField.set(null, mapperContext);
 
                 /**
                  * empty vertex value
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java
index f3ec40e..2087ea2 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java
@@ -44,17 +44,20 @@
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
+import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
 
 public class VertexFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private static final long serialVersionUID = 1L;
     private final IConfigurationFactory confFactory;
     private final IRecordDescriptorFactory inputRdFactory;
+    private final IRuntimeHookFactory preHookFactory;
 
     public VertexFileWriteOperatorDescriptor(JobSpecification spec, IConfigurationFactory confFactory,
-            IRecordDescriptorFactory inputRdFactory) {
+            IRecordDescriptorFactory inputRdFactory, IRuntimeHookFactory preHookFactory) {
         super(spec, 1, 0);
         this.confFactory = confFactory;
         this.inputRdFactory = inputRdFactory;
+        this.preHookFactory = preHookFactory;
     }
 
     @SuppressWarnings("rawtypes")
@@ -85,6 +88,8 @@
                 context = ctxFactory.createContext(conf, partition);
                 context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
                 try {
+                    if (preHookFactory != null)
+                        preHookFactory.createRuntimeHook().configure(ctx);
                     vertexWriter = outputFormat.createVertexWriter(context);
                 } catch (InterruptedException e) {
                     throw new HyracksDataException(e);
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PJobContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PJobContext.java
new file mode 100644
index 0000000..ceb085c
--- /dev/null
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PJobContext.java
@@ -0,0 +1,126 @@
+package edu.uci.ics.pregelix.dataflow.context;
+
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+
+public class PJobContext {
+    private static final Logger LOGGER = Logger.getLogger(RuntimeContext.class.getName());
+
+    private final Map<Long, List<FileReference>> iterationToFiles = new ConcurrentHashMap<Long, List<FileReference>>();
+    private final Map<TaskIterationID, IStateObject> appStateMap = new ConcurrentHashMap<TaskIterationID, IStateObject>();
+    private Long jobIdToSuperStep;
+    private Boolean jobIdToMove;
+
+    public void close() throws HyracksDataException {
+        for (Entry<Long, List<FileReference>> entry : iterationToFiles.entrySet())
+            for (FileReference fileRef : entry.getValue())
+                fileRef.delete();
+
+        iterationToFiles.clear();
+        appStateMap.clear();
+    }
+
+    public void clearState() throws HyracksDataException {
+        for (Entry<Long, List<FileReference>> entry : iterationToFiles.entrySet())
+            for (FileReference fileRef : entry.getValue())
+                fileRef.delete();
+
+        iterationToFiles.clear();
+        appStateMap.clear();
+    }
+
+    public Map<TaskIterationID, IStateObject> getAppStateStore() {
+        return appStateMap;
+    }
+
+    public static RuntimeContext get(IHyracksTaskContext ctx) {
+        return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
+    }
+
+    public void setVertexProperties(long numVertices, long numEdges, long currentIteration, ClassLoader cl) {
+        if (jobIdToMove == null || jobIdToMove == true) {
+            if (jobIdToSuperStep == null) {
+                if (currentIteration <= 0) {
+                    jobIdToSuperStep = 0L;
+                } else {
+                    jobIdToSuperStep = currentIteration;
+                }
+            }
+
+            long superStep = jobIdToSuperStep;
+            List<FileReference> files = iterationToFiles.remove(superStep - 1);
+            if (files != null) {
+                for (FileReference fileRef : files)
+                    fileRef.delete();
+            }
+
+            setProperties(numVertices, numEdges, currentIteration, superStep, false, cl);
+        }
+        System.gc();
+    }
+
+    public void recoverVertexProperties(long numVertices, long numEdges, long currentIteration, ClassLoader cl) {
+        if (jobIdToSuperStep == null) {
+            if (currentIteration <= 0) {
+                jobIdToSuperStep = 0L;
+            } else {
+                jobIdToSuperStep = currentIteration;
+            }
+        }
+
+        long superStep = jobIdToSuperStep;
+        List<FileReference> files = iterationToFiles.remove(superStep - 1);
+        if (files != null) {
+            for (FileReference fileRef : files)
+                fileRef.delete();
+        }
+
+        setProperties(numVertices, numEdges, currentIteration, superStep, true, cl);
+    }
+
+    public void endSuperStep() {
+        jobIdToMove = true;
+        LOGGER.info("end iteration " + Vertex.getSuperstep());
+    }
+
+    public Map<Long, List<FileReference>> getIterationToFiles() {
+        return iterationToFiles;
+    }
+
+    private void setProperties(long numVertices, long numEdges, long currentIteration, long superStep, boolean toMove,
+            ClassLoader cl) {
+        try {
+            Class<?> vClass = (Class<?>) cl.loadClass("edu.uci.ics.pregelix.api.graph.Vertex");
+            Method superStepMethod = vClass.getMethod("setSuperstep", Long.TYPE);
+            Method numVerticesMethod = vClass.getMethod("setNumVertices", Long.TYPE);
+            Method numEdgesMethod = vClass.getMethod("setNumEdges", Long.TYPE);
+
+            if (currentIteration > 0) {
+                //Vertex.setSuperstep(currentIteration);
+                superStepMethod.invoke(null, currentIteration);
+            } else {
+                //Vertex.setSuperstep(++superStep);
+                superStepMethod.invoke(null, ++superStep);
+            }
+            //Vertex.setNumVertices(numVertices);
+            numVerticesMethod.invoke(null, numVertices);
+            //Vertex.setNumEdges(numEdges);
+            numEdgesMethod.invoke(null, numEdges);
+            jobIdToSuperStep = superStep;
+            jobIdToMove = toMove;
+            LOGGER.info("start iteration " + Vertex.getSuperstep());
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index f3f7513..854e3dc 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -20,7 +20,6 @@
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadFactory;
-import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -48,7 +47,6 @@
 import edu.uci.ics.pregelix.api.graph.Vertex;
 
 public class RuntimeContext implements IWorkspaceFileFactory {
-    private static final Logger LOGGER = Logger.getLogger(RuntimeContext.class.getName());
 
     private final IIndexLifecycleManager lcManager;
     private final ILocalResourceRepository localResourceRepository;
@@ -57,10 +55,7 @@
     private final List<IVirtualBufferCache> vbcs;
     private final IFileMapManager fileMapManager;
     private final IOManager ioManager;
-    private final Map<Long, List<FileReference>> iterationToFiles = new ConcurrentHashMap<Long, List<FileReference>>();
-    private final Map<StateKey, IStateObject> appStateMap = new ConcurrentHashMap<StateKey, IStateObject>();
-    private final Map<String, Long> jobIdToSuperStep = new ConcurrentHashMap<String, Long>();
-    private final Map<String, Boolean> jobIdToMove = new ConcurrentHashMap<String, Boolean>();
+    private final Map<String, PJobContext> activeJobs = new ConcurrentHashMap<String, PJobContext>();
 
     private final ThreadFactory threadFactory = new ThreadFactory() {
         public Thread newThread(Runnable r) {
@@ -91,28 +86,12 @@
         resourceIdFactory = new ResourceIdFactory(0);
     }
 
-    public void close() throws HyracksDataException {
-        for (Entry<Long, List<FileReference>> entry : iterationToFiles.entrySet())
-            for (FileReference fileRef : entry.getValue())
-                fileRef.delete();
-
-        iterationToFiles.clear();
+    public synchronized void close() throws HyracksDataException {
         bufferCache.close();
-        appStateMap.clear();
-
-        System.gc();
-    }
-
-    public void clearState(String jobId) throws HyracksDataException {
-        for (Entry<Long, List<FileReference>> entry : iterationToFiles.entrySet())
-            for (FileReference fileRef : entry.getValue())
-                fileRef.delete();
-
-        iterationToFiles.clear();
-        appStateMap.clear();
-        jobIdToMove.remove(jobId);
-        jobIdToSuperStep.remove(jobId);
-        System.gc();
+        for (Entry<String, PJobContext> entry : activeJobs.entrySet()) {
+            entry.getValue().close();
+        }
+        activeJobs.clear();
     }
 
     public ILocalResourceRepository getLocalResourceRepository() {
@@ -139,87 +118,55 @@
         return fileMapManager;
     }
 
-    public Map<StateKey, IStateObject> getAppStateStore() {
-        return appStateMap;
+    public synchronized Map<TaskIterationID, IStateObject> getAppStateStore(String jobId) {
+        PJobContext activeJob = getActiveJob(jobId);
+        return activeJob.getAppStateStore();
     }
 
     public static RuntimeContext get(IHyracksTaskContext ctx) {
         return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
     }
 
-    public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, long currentIteration) {
-        Boolean toMove = jobIdToMove.get(jobId);
-        if (toMove == null || toMove == true) {
-            if (jobIdToSuperStep.get(jobId) == null) {
-                if (currentIteration <= 0) {
-                    jobIdToSuperStep.put(jobId, 0L);
-                } else {
-                    jobIdToSuperStep.put(jobId, currentIteration);
-                }
-            }
-
-            long superStep = jobIdToSuperStep.get(jobId);
-            List<FileReference> files = iterationToFiles.remove(superStep - 1);
-            if (files != null) {
-                for (FileReference fileRef : files)
-                    fileRef.delete();
-            }
-
-            if (currentIteration > 0) {
-                Vertex.setSuperstep(currentIteration);
-            } else {
-                Vertex.setSuperstep(++superStep);
-            }
-            Vertex.setNumVertices(numVertices);
-            Vertex.setNumEdges(numEdges);
-            jobIdToSuperStep.put(jobId, superStep);
-            jobIdToMove.put(jobId, false);
-            LOGGER.info("start iteration " + Vertex.getSuperstep());
-        }
-        System.gc();
+    public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, long currentIteration,
+            ClassLoader cl) {
+        PJobContext activeJob = getActiveJob(jobId);
+        activeJob.setVertexProperties(numVertices, numEdges, currentIteration, cl);
     }
 
     public synchronized void recoverVertexProperties(String jobId, long numVertices, long numEdges,
-            long currentIteration) {
-        if (jobIdToSuperStep.get(jobId) == null) {
-            if (currentIteration <= 0) {
-                jobIdToSuperStep.put(jobId, 0L);
-            } else {
-                jobIdToSuperStep.put(jobId, currentIteration);
-            }
-        }
-
-        long superStep = jobIdToSuperStep.get(jobId);
-        List<FileReference> files = iterationToFiles.remove(superStep - 1);
-        if (files != null) {
-            for (FileReference fileRef : files)
-                fileRef.delete();
-        }
-
-        if (currentIteration > 0) {
-            Vertex.setSuperstep(currentIteration);
-        } else {
-            Vertex.setSuperstep(++superStep);
-        }
-        Vertex.setNumVertices(numVertices);
-        Vertex.setNumEdges(numEdges);
-        jobIdToSuperStep.put(jobId, superStep);
-        jobIdToMove.put(jobId, true);
-        LOGGER.info("recovered iteration " + Vertex.getSuperstep());
+            long currentIteration, ClassLoader cl) {
+        PJobContext activeJob = getActiveJob(jobId);
+        activeJob.recoverVertexProperties(numVertices, numEdges, currentIteration, cl);
     }
 
-    public synchronized void endSuperStep(String pregelixJobId) {
-        jobIdToMove.put(pregelixJobId, true);
-        LOGGER.info("end iteration " + Vertex.getSuperstep());
+    public synchronized void endSuperStep(String jobId) {
+        PJobContext activeJob = getActiveJob(jobId);
+        activeJob.endSuperStep();
+    }
+
+    public synchronized void clearState(String jobId) throws HyracksDataException {
+        PJobContext activeJob = getActiveJob(jobId);
+        activeJob.clearState();
+        activeJobs.remove(jobId);
+    }
+
+    private PJobContext getActiveJob(String jobId) {
+        PJobContext activeJob = activeJobs.get(jobId);
+        if (activeJob == null) {
+            activeJob = new PJobContext();
+            activeJobs.put(jobId, activeJob);
+        }
+        return activeJob;
     }
 
     @Override
-    public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
-        final FileReference fRef = ioManager.createWorkspaceFile(prefix);
-        List<FileReference> files = iterationToFiles.get(Vertex.getSuperstep());
+    public FileReference createManagedWorkspaceFile(String jobId) throws HyracksDataException {
+        final FileReference fRef = ioManager.createWorkspaceFile(jobId);
+        PJobContext activeJob = getActiveJob(jobId);
+        List<FileReference> files = activeJob.getIterationToFiles().get(Vertex.getSuperstep());
         if (files == null) {
             files = new ArrayList<FileReference>();
-            iterationToFiles.put(Vertex.getSuperstep(), files);
+            activeJob.getIterationToFiles().put(Vertex.getSuperstep(), files);
         }
         files.add(fRef);
         return fRef;
@@ -229,4 +176,5 @@
     public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
         return ioManager.createWorkspaceFile(prefix);
     }
+
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/StateKey.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskID.java
similarity index 64%
rename from pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/StateKey.java
rename to pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskID.java
index cbd90b9..f219ed2 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/StateKey.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskID.java
@@ -12,34 +12,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package edu.uci.ics.pregelix.dataflow.context;
 
-import edu.uci.ics.hyracks.api.job.JobId;
+public class TaskID {
 
-public class StateKey {
-    private final JobId jobId;
-    private final int partition;
+    private String jobId;
+    private int partition;
 
-    public StateKey(JobId jobId, int partition) {
+    public TaskID(String jobId, int partition) {
         this.jobId = jobId;
         this.partition = partition;
     }
 
+    public String getJobId() {
+        return jobId;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
     @Override
     public int hashCode() {
-        return jobId.hashCode() * partition;
+        return jobId.hashCode() + partition;
     }
 
     @Override
     public boolean equals(Object o) {
-        if (!(o instanceof StateKey))
+        if (!(o instanceof TaskID)) {
             return false;
-        StateKey key = (StateKey) o;
-        return key.jobId.equals(jobId) && key.partition == partition;
+        }
+        TaskID tid = (TaskID) o;
+        return jobId.equals(tid.getJobId()) && partition == tid.getPartition();
     }
 
     @Override
     public String toString() {
-        return jobId.toString() + ":" + partition;
+        return "job:" + jobId + " partition:" + partition;
     }
+
 }
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskIterationID.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskIterationID.java
new file mode 100644
index 0000000..53c6a0c
--- /dev/null
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskIterationID.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.pregelix.dataflow.context;
+
+public class TaskIterationID {
+
+    private TaskID tid;
+    private int iteration;
+
+    public TaskIterationID(TaskID tid, int iteration) {
+        this.tid = tid;
+        this.iteration = iteration;
+    }
+
+    public TaskIterationID(String jobId, int partition, int iteration) {
+        this.tid = new TaskID(jobId, partition);
+        this.iteration = iteration;
+    }
+
+    public TaskID getTaskID() {
+        return tid;
+    }
+
+    public int getIteration() {
+        return iteration;
+    }
+
+    public TaskIterationID getNextTaskIterationID() {
+        return new TaskIterationID(tid, iteration + 1);
+    }
+
+    public TaskIterationID getPreviousTaskIterationID() {
+        return new TaskIterationID(tid, iteration - 1);
+    }
+
+    @Override
+    public int hashCode() {
+        return tid.hashCode() + iteration;
+    }
+
+    public boolean equals(Object o) {
+        if (!(o instanceof TaskIterationID)) {
+            return false;
+        }
+        TaskIterationID tiid = (TaskIterationID) o;
+        return tid.equals(tiid.getTaskID()) && iteration == tiid.getIteration();
+    }
+
+    @Override
+    public String toString() {
+        return tid.toString() + " iteration:" + iteration;
+    }
+}
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
index 02097bf..8052b7c 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
@@ -28,73 +28,66 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.api.util.JobStateUtils;
 import edu.uci.ics.pregelix.dataflow.context.RuntimeContext;
-import edu.uci.ics.pregelix.dataflow.context.StateKey;
+import edu.uci.ics.pregelix.dataflow.context.TaskIterationID;
 
 public class IterationUtils {
     public static final String TMP_DIR = "/tmp/";
 
-    public static void setIterationState(IHyracksTaskContext ctx, int partition, IStateObject state) {
+    public static void setIterationState(IHyracksTaskContext ctx, String pregelixJobId, int partition, int iteration,
+            IStateObject state) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
         RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
-        Map<StateKey, IStateObject> map = context.getAppStateStore();
-        map.put(new StateKey(ctx.getJobletContext().getJobId(), partition), state);
+        Map<TaskIterationID, IStateObject> map = context.getAppStateStore(pregelixJobId);
+        map.put(new TaskIterationID(pregelixJobId, partition, iteration), state);
     }
 
-    public static IStateObject getIterationState(IHyracksTaskContext ctx, int partition) {
-        JobId currentId = ctx.getJobletContext().getJobId();
-        JobId lastId = new JobId(currentId.getId() - 1);
+    public static IStateObject getIterationState(IHyracksTaskContext ctx, String pregelixJobId, int partition,
+            int iteration) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
         RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
-        Map<StateKey, IStateObject> map = context.getAppStateStore();
-        IStateObject state = map.get(new StateKey(lastId, partition));
-        while (state == null) {
-            /** in case the last job is a checkpointing job */
-            lastId = new JobId(lastId.getId() - 1);
-            state = map.get(new StateKey(lastId, partition));
-        }
+        Map<TaskIterationID, IStateObject> map = context.getAppStateStore(pregelixJobId);
+        IStateObject state = map.get(new TaskIterationID(pregelixJobId, partition, iteration));
         return state;
     }
 
-    public static void removeIterationState(IHyracksTaskContext ctx, int partition) {
-        JobId currentId = ctx.getJobletContext().getJobId();
-        JobId lastId = new JobId(currentId.getId() - 1);
+    public static void removeIterationState(IHyracksTaskContext ctx, String pregelixJobId, int partition, int iteration) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
         RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
-        Map<StateKey, IStateObject> map = context.getAppStateStore();
-        map.remove(new StateKey(lastId, partition));
+        Map<TaskIterationID, IStateObject> map = context.getAppStateStore(pregelixJobId);
+        map.remove(new TaskIterationID(pregelixJobId, partition, iteration));
     }
 
-    public static void endSuperStep(String giraphJobId, IHyracksTaskContext ctx) {
+    public static void endSuperStep(String pregelixJobId, IHyracksTaskContext ctx) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
         RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
-        context.endSuperStep(giraphJobId);
+        context.endSuperStep(pregelixJobId);
     }
 
-    public static void setProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, long currentIteration) {
-        INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
-        RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
-        context.setVertexProperties(jobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
-                conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration);
-    }
-
-    public static void recoverProperties(String jobId, IHyracksTaskContext ctx, Configuration conf,
+    public static void setProperties(String pregelixJobId, IHyracksTaskContext ctx, Configuration conf,
             long currentIteration) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
         RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
-        context.recoverVertexProperties(jobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
-                conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration);
+        context.setVertexProperties(pregelixJobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
+                conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration, ctx.getJobletContext().getClassLoader());
     }
 
-    public static void writeTerminationState(Configuration conf, String jobId, boolean terminate)
+    public static void recoverProperties(String pregelixJobId, IHyracksTaskContext ctx, Configuration conf,
+            long currentIteration) {
+        INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
+        RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
+        context.recoverVertexProperties(pregelixJobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
+                conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration, ctx.getJobletContext().getClassLoader());
+    }
+
+    public static void writeTerminationState(Configuration conf, String pregelixJobId, boolean terminate)
             throws HyracksDataException {
         try {
             FileSystem dfs = FileSystem.get(conf);
-            String pathStr = IterationUtils.TMP_DIR + jobId;
+            String pathStr = IterationUtils.TMP_DIR + pregelixJobId;
             Path path = new Path(pathStr);
             FSDataOutputStream output = dfs.create(path, true);
             output.writeBoolean(terminate);
@@ -105,11 +98,11 @@
         }
     }
 
-    public static void writeGlobalAggregateValue(Configuration conf, String jobId, Writable agg)
+    public static void writeGlobalAggregateValue(Configuration conf, String pregelixJobId, Writable agg)
             throws HyracksDataException {
         try {
             FileSystem dfs = FileSystem.get(conf);
-            String pathStr = IterationUtils.TMP_DIR + jobId + "agg";
+            String pathStr = IterationUtils.TMP_DIR + pregelixJobId + "agg";
             Path path = new Path(pathStr);
             FSDataOutputStream output = dfs.create(path, true);
             agg.write(output);
@@ -120,10 +113,10 @@
         }
     }
 
-    public static boolean readTerminationState(Configuration conf, String jobId) throws HyracksDataException {
+    public static boolean readTerminationState(Configuration conf, String pregelixJobId) throws HyracksDataException {
         try {
             FileSystem dfs = FileSystem.get(conf);
-            String pathStr = IterationUtils.TMP_DIR + jobId;
+            String pathStr = IterationUtils.TMP_DIR + pregelixJobId;
             Path path = new Path(pathStr);
             FSDataInputStream input = dfs.open(path);
             boolean terminate = input.readBoolean();
@@ -134,8 +127,8 @@
         }
     }
 
-    public static void writeForceTerminationState(Configuration conf, String jobId) throws HyracksDataException {
-        JobStateUtils.writeForceTerminationState(conf, jobId);
+    public static void writeForceTerminationState(Configuration conf, String pregelixJobId) throws HyracksDataException {
+        JobStateUtils.writeForceTerminationState(conf, pregelixJobId);
     }
 
     public static boolean readForceTerminationState(Configuration conf, String jobId) throws HyracksDataException {
diff --git a/pregelix/pregelix-example/data/update/webmap_link.txt b/pregelix/pregelix-example/data/update/webmap_link.txt
new file mode 100644
index 0000000..9812045
--- /dev/null
+++ b/pregelix/pregelix-example/data/update/webmap_link.txt
@@ -0,0 +1,10000 @@
+1
+2
+3
+4
+5
+6
+7
+8
+9
+10
+11
+12
+13
+14
+15
+16
+17
+18
+19
+20
+21
+22
+23
+24
+25
+26
+27
+28
+29
+30
+31
+32
+33
+34
+35
+36
+37
+38
+39
+40
+41
+42
+43
+44
+45
+46
+47
+48
+49
+50
+51
+52
+53
+54
+55
+56
+57
+58
+59
+60
+61
+62
+63
+64
+65
+66
+67
+68
+69
+70
+71
+72
+73
+74
+75
+76
+77
+78
+79
+80
+81
+82
+83
+84
+85
+86
+87
+88
+89
+90
+91
+92
+93
+94
+95
+96
+97
+98
+99
+100
+101
+102
+103
+104
+105
+106
+107
+108
+109
+110
+111
+112
+113
+114
+115
+116
+117
+118
+119
+120
+121
+122
+123
+124
+125
+126
+127
+128
+129
+130
+131
+132
+133
+134
+135
+136
+137
+138
+139
+140
+141
+142
+143
+144
+145
+146
+147
+148
+149
+150
+151
+152
+153
+154
+155
+156
+157
+158
+159
+160
+161
+162
+163
+164
+165
+166
+167
+168
+169
+170
+171
+172
+173
+174
+175
+176
+177
+178
+179
+180
+181
+182
+183
+184
+185
+186
+187
+188
+189
+190
+191
+192
+193
+194
+195
+196
+197
+198
+199
+200
+201
+202
+203
+204
+205
+206
+207
+208
+209
+210
+211
+212
+213
+214
+215
+216
+217
+218
+219
+220
+221
+222
+223
+224
+225
+226
+227
+228
+229
+230
+231
+232
+233
+234
+235
+236
+237
+238
+239
+240
+241
+242
+243
+244
+245
+246
+247
+248
+249
+250
+251
+252
+253
+254
+255
+256
+257
+258
+259
+260
+261
+262
+263
+264
+265
+266
+267
+268
+269
+270
+271
+272
+273
+274
+275
+276
+277
+278
+279
+280
+281
+282
+283
+284
+285
+286
+287
+288
+289
+290
+291
+292
+293
+294
+295
+296
+297
+298
+299
+300
+301
+302
+303
+304
+305
+306
+307
+308
+309
+310
+311
+312
+313
+314
+315
+316
+317
+318
+319
+320
+321
+322
+323
+324
+325
+326
+327
+328
+329
+330
+331
+332
+333
+334
+335
+336
+337
+338
+339
+340
+341
+342
+343
+344
+345
+346
+347
+348
+349
+350
+351
+352
+353
+354
+355
+356
+357
+358
+359
+360
+361
+362
+363
+364
+365
+366
+367
+368
+369
+370
+371
+372
+373
+374
+375
+376
+377
+378
+379
+380
+381
+382
+383
+384
+385
+386
+387
+388
+389
+390
+391
+392
+393
+394
+395
+396
+397
+398
+399
+400
+401
+402
+403
+404
+405
+406
+407
+408
+409
+410
+411
+412
+413
+414
+415
+416
+417
+418
+419
+420
+421
+422
+423
+424
+425
+426
+427
+428
+429
+430
+431
+432
+433
+434
+435
+436
+437
+438
+439
+440
+441
+442
+443
+444
+445
+446
+447
+448
+449
+450
+451
+452
+453
+454
+455
+456
+457
+458
+459
+460
+461
+462
+463
+464
+465
+466
+467
+468
+469
+470
+471
+472
+473
+474
+475
+476
+477
+478
+479
+480
+481
+482
+483
+484
+485
+486
+487
+488
+489
+490
+491
+492
+493
+494
+495
+496
+497
+498
+499
+500
+501
+502
+503
+504
+505
+506
+507
+508
+509
+510
+511
+512
+513
+514
+515
+516
+517
+518
+519
+520
+521
+522
+523
+524
+525
+526
+527
+528
+529
+530
+531
+532
+533
+534
+535
+536
+537
+538
+539
+540
+541
+542
+543
+544
+545
+546
+547
+548
+549
+550
+551
+552
+553
+554
+555
+556
+557
+558
+559
+560
+561
+562
+563
+564
+565
+566
+567
+568
+569
+570
+571
+572
+573
+574
+575
+576
+577
+578
+579
+580
+581
+582
+583
+584
+585
+586
+587
+588
+589
+590
+591
+592
+593
+594
+595
+596
+597
+598
+599
+600
+601
+602
+603
+604
+605
+606
+607
+608
+609
+610
+611
+612
+613
+614
+615
+616
+617
+618
+619
+620
+621
+622
+623
+624
+625
+626
+627
+628
+629
+630
+631
+632
+633
+634
+635
+636
+637
+638
+639
+640
+641
+642
+643
+644
+645
+646
+647
+648
+649
+650
+651
+652
+653
+654
+655
+656
+657
+658
+659
+660
+661
+662
+663
+664
+665
+666
+667
+668
+669
+670
+671
+672
+673
+674
+675
+676
+677
+678
+679
+680
+681
+682
+683
+684
+685
+686
+687
+688
+689
+690
+691
+692
+693
+694
+695
+696
+697
+698
+699
+700
+701
+702
+703
+704
+705
+706
+707
+708
+709
+710
+711
+712
+713
+714
+715
+716
+717
+718
+719
+720
+721
+722
+723
+724
+725
+726
+727
+728
+729
+730
+731
+732
+733
+734
+735
+736
+737
+738
+739
+740
+741
+742
+743
+744
+745
+746
+747
+748
+749
+750
+751
+752
+753
+754
+755
+756
+757
+758
+759
+760
+761
+762
+763
+764
+765
+766
+767
+768
+769
+770
+771
+772
+773
+774
+775
+776
+777
+778
+779
+780
+781
+782
+783
+784
+785
+786
+787
+788
+789
+790
+791
+792
+793
+794
+795
+796
+797
+798
+799
+800
+801
+802
+803
+804
+805
+806
+807
+808
+809
+810
+811
+812
+813
+814
+815
+816
+817
+818
+819
+820
+821
+822
+823
+824
+825
+826
+827
+828
+829
+830
+831
+832
+833
+834
+835
+836
+837
+838
+839
+840
+841
+842
+843
+844
+845
+846
+847
+848
+849
+850
+851
+852
+853
+854
+855
+856
+857
+858
+859
+860
+861
+862
+863
+864
+865
+866
+867
+868
+869
+870
+871
+872
+873
+874
+875
+876
+877
+878
+879
+880
+881
+882
+883
+884
+885
+886
+887
+888
+889
+890
+891
+892
+893
+894
+895
+896
+897
+898
+899
+900
+901
+902
+903
+904
+905
+906
+907
+908
+909
+910
+911
+912
+913
+914
+915
+916
+917
+918
+919
+920
+921
+922
+923
+924
+925
+926
+927
+928
+929
+930
+931
+932
+933
+934
+935
+936
+937
+938
+939
+940
+941
+942
+943
+944
+945
+946
+947
+948
+949
+950
+951
+952
+953
+954
+955
+956
+957
+958
+959
+960
+961
+962
+963
+964
+965
+966
+967
+968
+969
+970
+971
+972
+973
+974
+975
+976
+977
+978
+979
+980
+981
+982
+983
+984
+985
+986
+987
+988
+989
+990
+991
+992
+993
+994
+995
+996
+997
+998
+999
+1000
+1001
+1002
+1003
+1004
+1005
+1006
+1007
+1008
+1009
+1010
+1011
+1012
+1013
+1014
+1015
+1016
+1017
+1018
+1019
+1020
+1021
+1022
+1023
+1024
+1025
+1026
+1027
+1028
+1029
+1030
+1031
+1032
+1033
+1034
+1035
+1036
+1037
+1038
+1039
+1040
+1041
+1042
+1043
+1044
+1045
+1046
+1047
+1048
+1049
+1050
+1051
+1052
+1053
+1054
+1055
+1056
+1057
+1058
+1059
+1060
+1061
+1062
+1063
+1064
+1065
+1066
+1067
+1068
+1069
+1070
+1071
+1072
+1073
+1074
+1075
+1076
+1077
+1078
+1079
+1080
+1081
+1082
+1083
+1084
+1085
+1086
+1087
+1088
+1089
+1090
+1091
+1092
+1093
+1094
+1095
+1096
+1097
+1098
+1099
+1100
+1101
+1102
+1103
+1104
+1105
+1106
+1107
+1108
+1109
+1110
+1111
+1112
+1113
+1114
+1115
+1116
+1117
+1118
+1119
+1120
+1121
+1122
+1123
+1124
+1125
+1126
+1127
+1128
+1129
+1130
+1131
+1132
+1133
+1134
+1135
+1136
+1137
+1138
+1139
+1140
+1141
+1142
+1143
+1144
+1145
+1146
+1147
+1148
+1149
+1150
+1151
+1152
+1153
+1154
+1155
+1156
+1157
+1158
+1159
+1160
+1161
+1162
+1163
+1164
+1165
+1166
+1167
+1168
+1169
+1170
+1171
+1172
+1173
+1174
+1175
+1176
+1177
+1178
+1179
+1180
+1181
+1182
+1183
+1184
+1185
+1186
+1187
+1188
+1189
+1190
+1191
+1192
+1193
+1194
+1195
+1196
+1197
+1198
+1199
+1200
+1201
+1202
+1203
+1204
+1205
+1206
+1207
+1208
+1209
+1210
+1211
+1212
+1213
+1214
+1215
+1216
+1217
+1218
+1219
+1220
+1221
+1222
+1223
+1224
+1225
+1226
+1227
+1228
+1229
+1230
+1231
+1232
+1233
+1234
+1235
+1236
+1237
+1238
+1239
+1240
+1241
+1242
+1243
+1244
+1245
+1246
+1247
+1248
+1249
+1250
+1251
+1252
+1253
+1254
+1255
+1256
+1257
+1258
+1259
+1260
+1261
+1262
+1263
+1264
+1265
+1266
+1267
+1268
+1269
+1270
+1271
+1272
+1273
+1274
+1275
+1276
+1277
+1278
+1279
+1280
+1281
+1282
+1283
+1284
+1285
+1286
+1287
+1288
+1289
+1290
+1291
+1292
+1293
+1294
+1295
+1296
+1297
+1298
+1299
+1300
+1301
+1302
+1303
+1304
+1305
+1306
+1307
+1308
+1309
+1310
+1311
+1312
+1313
+1314
+1315
+1316
+1317
+1318
+1319
+1320
+1321
+1322
+1323
+1324
+1325
+1326
+1327
+1328
+1329
+1330
+1331
+1332
+1333
+1334
+1335
+1336
+1337
+1338
+1339
+1340
+1341
+1342
+1343
+1344
+1345
+1346
+1347
+1348
+1349
+1350
+1351
+1352
+1353
+1354
+1355
+1356
+1357
+1358
+1359
+1360
+1361
+1362
+1363
+1364
+1365
+1366
+1367
+1368
+1369
+1370
+1371
+1372
+1373
+1374
+1375
+1376
+1377
+1378
+1379
+1380
+1381
+1382
+1383
+1384
+1385
+1386
+1387
+1388
+1389
+1390
+1391
+1392
+1393
+1394
+1395
+1396
+1397
+1398
+1399
+1400
+1401
+1402
+1403
+1404
+1405
+1406
+1407
+1408
+1409
+1410
+1411
+1412
+1413
+1414
+1415
+1416
+1417
+1418
+1419
+1420
+1421
+1422
+1423
+1424
+1425
+1426
+1427
+1428
+1429
+1430
+1431
+1432
+1433
+1434
+1435
+1436
+1437
+1438
+1439
+1440
+1441
+1442
+1443
+1444
+1445
+1446
+1447
+1448
+1449
+1450
+1451
+1452
+1453
+1454
+1455
+1456
+1457
+1458
+1459
+1460
+1461
+1462
+1463
+1464
+1465
+1466
+1467
+1468
+1469
+1470
+1471
+1472
+1473
+1474
+1475
+1476
+1477
+1478
+1479
+1480
+1481
+1482
+1483
+1484
+1485
+1486
+1487
+1488
+1489
+1490
+1491
+1492
+1493
+1494
+1495
+1496
+1497
+1498
+1499
+1500
+1501
+1502
+1503
+1504
+1505
+1506
+1507
+1508
+1509
+1510
+1511
+1512
+1513
+1514
+1515
+1516
+1517
+1518
+1519
+1520
+1521
+1522
+1523
+1524
+1525
+1526
+1527
+1528
+1529
+1530
+1531
+1532
+1533
+1534
+1535
+1536
+1537
+1538
+1539
+1540
+1541
+1542
+1543
+1544
+1545
+1546
+1547
+1548
+1549
+1550
+1551
+1552
+1553
+1554
+1555
+1556
+1557
+1558
+1559
+1560
+1561
+1562
+1563
+1564
+1565
+1566
+1567
+1568
+1569
+1570
+1571
+1572
+1573
+1574
+1575
+1576
+1577
+1578
+1579
+1580
+1581
+1582
+1583
+1584
+1585
+1586
+1587
+1588
+1589
+1590
+1591
+1592
+1593
+1594
+1595
+1596
+1597
+1598
+1599
+1600
+1601
+1602
+1603
+1604
+1605
+1606
+1607
+1608
+1609
+1610
+1611
+1612
+1613
+1614
+1615
+1616
+1617
+1618
+1619
+1620
+1621
+1622
+1623
+1624
+1625
+1626
+1627
+1628
+1629
+1630
+1631
+1632
+1633
+1634
+1635
+1636
+1637
+1638
+1639
+1640
+1641
+1642
+1643
+1644
+1645
+1646
+1647
+1648
+1649
+1650
+1651
+1652
+1653
+1654
+1655
+1656
+1657
+1658
+1659
+1660
+1661
+1662
+1663
+1664
+1665
+1666
+1667
+1668
+1669
+1670
+1671
+1672
+1673
+1674
+1675
+1676
+1677
+1678
+1679
+1680
+1681
+1682
+1683
+1684
+1685
+1686
+1687
+1688
+1689
+1690
+1691
+1692
+1693
+1694
+1695
+1696
+1697
+1698
+1699
+1700
+1701
+1702
+1703
+1704
+1705
+1706
+1707
+1708
+1709
+1710
+1711
+1712
+1713
+1714
+1715
+1716
+1717
+1718
+1719
+1720
+1721
+1722
+1723
+1724
+1725
+1726
+1727
+1728
+1729
+1730
+1731
+1732
+1733
+1734
+1735
+1736
+1737
+1738
+1739
+1740
+1741
+1742
+1743
+1744
+1745
+1746
+1747
+1748
+1749
+1750
+1751
+1752
+1753
+1754
+1755
+1756
+1757
+1758
+1759
+1760
+1761
+1762
+1763
+1764
+1765
+1766
+1767
+1768
+1769
+1770
+1771
+1772
+1773
+1774
+1775
+1776
+1777
+1778
+1779
+1780
+1781
+1782
+1783
+1784
+1785
+1786
+1787
+1788
+1789
+1790
+1791
+1792
+1793
+1794
+1795
+1796
+1797
+1798
+1799
+1800
+1801
+1802
+1803
+1804
+1805
+1806
+1807
+1808
+1809
+1810
+1811
+1812
+1813
+1814
+1815
+1816
+1817
+1818
+1819
+1820
+1821
+1822
+1823
+1824
+1825
+1826
+1827
+1828
+1829
+1830
+1831
+1832
+1833
+1834
+1835
+1836
+1837
+1838
+1839
+1840
+1841
+1842
+1843
+1844
+1845
+1846
+1847
+1848
+1849
+1850
+1851
+1852
+1853
+1854
+1855
+1856
+1857
+1858
+1859
+1860
+1861
+1862
+1863
+1864
+1865
+1866
+1867
+1868
+1869
+1870
+1871
+1872
+1873
+1874
+1875
+1876
+1877
+1878
+1879
+1880
+1881
+1882
+1883
+1884
+1885
+1886
+1887
+1888
+1889
+1890
+1891
+1892
+1893
+1894
+1895
+1896
+1897
+1898
+1899
+1900
+1901
+1902
+1903
+1904
+1905
+1906
+1907
+1908
+1909
+1910
+1911
+1912
+1913
+1914
+1915
+1916
+1917
+1918
+1919
+1920
+1921
+1922
+1923
+1924
+1925
+1926
+1927
+1928
+1929
+1930
+1931
+1932
+1933
+1934
+1935
+1936
+1937
+1938
+1939
+1940
+1941
+1942
+1943
+1944
+1945
+1946
+1947
+1948
+1949
+1950
+1951
+1952
+1953
+1954
+1955
+1956
+1957
+1958
+1959
+1960
+1961
+1962
+1963
+1964
+1965
+1966
+1967
+1968
+1969
+1970
+1971
+1972
+1973
+1974
+1975
+1976
+1977
+1978
+1979
+1980
+1981
+1982
+1983
+1984
+1985
+1986
+1987
+1988
+1989
+1990
+1991
+1992
+1993
+1994
+1995
+1996
+1997
+1998
+1999
+2000
+2001
+2002
+2003
+2004
+2005
+2006
+2007
+2008
+2009
+2010
+2011
+2012
+2013
+2014
+2015
+2016
+2017
+2018
+2019
+2020
+2021
+2022
+2023
+2024
+2025
+2026
+2027
+2028
+2029
+2030
+2031
+2032
+2033
+2034
+2035
+2036
+2037
+2038
+2039
+2040
+2041
+2042
+2043
+2044
+2045
+2046
+2047
+2048
+2049
+2050
+2051
+2052
+2053
+2054
+2055
+2056
+2057
+2058
+2059
+2060
+2061
+2062
+2063
+2064
+2065
+2066
+2067
+2068
+2069
+2070
+2071
+2072
+2073
+2074
+2075
+2076
+2077
+2078
+2079
+2080
+2081
+2082
+2083
+2084
+2085
+2086
+2087
+2088
+2089
+2090
+2091
+2092
+2093
+2094
+2095
+2096
+2097
+2098
+2099
+2100
+2101
+2102
+2103
+2104
+2105
+2106
+2107
+2108
+2109
+2110
+2111
+2112
+2113
+2114
+2115
+2116
+2117
+2118
+2119
+2120
+2121
+2122
+2123
+2124
+2125
+2126
+2127
+2128
+2129
+2130
+2131
+2132
+2133
+2134
+2135
+2136
+2137
+2138
+2139
+2140
+2141
+2142
+2143
+2144
+2145
+2146
+2147
+2148
+2149
+2150
+2151
+2152
+2153
+2154
+2155
+2156
+2157
+2158
+2159
+2160
+2161
+2162
+2163
+2164
+2165
+2166
+2167
+2168
+2169
+2170
+2171
+2172
+2173
+2174
+2175
+2176
+2177
+2178
+2179
+2180
+2181
+2182
+2183
+2184
+2185
+2186
+2187
+2188
+2189
+2190
+2191
+2192
+2193
+2194
+2195
+2196
+2197
+2198
+2199
+2200
+2201
+2202
+2203
+2204
+2205
+2206
+2207
+2208
+2209
+2210
+2211
+2212
+2213
+2214
+2215
+2216
+2217
+2218
+2219
+2220
+2221
+2222
+2223
+2224
+2225
+2226
+2227
+2228
+2229
+2230
+2231
+2232
+2233
+2234
+2235
+2236
+2237
+2238
+2239
+2240
+2241
+2242
+2243
+2244
+2245
+2246
+2247
+2248
+2249
+2250
+2251
+2252
+2253
+2254
+2255
+2256
+2257
+2258
+2259
+2260
+2261
+2262
+2263
+2264
+2265
+2266
+2267
+2268
+2269
+2270
+2271
+2272
+2273
+2274
+2275
+2276
+2277
+2278
+2279
+2280
+2281
+2282
+2283
+2284
+2285
+2286
+2287
+2288
+2289
+2290
+2291
+2292
+2293
+2294
+2295
+2296
+2297
+2298
+2299
+2300
+2301
+2302
+2303
+2304
+2305
+2306
+2307
+2308
+2309
+2310
+2311
+2312
+2313
+2314
+2315
+2316
+2317
+2318
+2319
+2320
+2321
+2322
+2323
+2324
+2325
+2326
+2327
+2328
+2329
+2330
+2331
+2332
+2333
+2334
+2335
+2336
+2337
+2338
+2339
+2340
+2341
+2342
+2343
+2344
+2345
+2346
+2347
+2348
+2349
+2350
+2351
+2352
+2353
+2354
+2355
+2356
+2357
+2358
+2359
+2360
+2361
+2362
+2363
+2364
+2365
+2366
+2367
+2368
+2369
+2370
+2371
+2372
+2373
+2374
+2375
+2376
+2377
+2378
+2379
+2380
+2381
+2382
+2383
+2384
+2385
+2386
+2387
+2388
+2389
+2390
+2391
+2392
+2393
+2394
+2395
+2396
+2397
+2398
+2399
+2400
+2401
+2402
+2403
+2404
+2405
+2406
+2407
+2408
+2409
+2410
+2411
+2412
+2413
+2414
+2415
+2416
+2417
+2418
+2419
+2420
+2421
+2422
+2423
+2424
+2425
+2426
+2427
+2428
+2429
+2430
+2431
+2432
+2433
+2434
+2435
+2436
+2437
+2438
+2439
+2440
+2441
+2442
+2443
+2444
+2445
+2446
+2447
+2448
+2449
+2450
+2451
+2452
+2453
+2454
+2455
+2456
+2457
+2458
+2459
+2460
+2461
+2462
+2463
+2464
+2465
+2466
+2467
+2468
+2469
+2470
+2471
+2472
+2473
+2474
+2475
+2476
+2477
+2478
+2479
+2480
+2481
+2482
+2483
+2484
+2485
+2486
+2487
+2488
+2489
+2490
+2491
+2492
+2493
+2494
+2495
+2496
+2497
+2498
+2499
+2500
+2501
+2502
+2503
+2504
+2505
+2506
+2507
+2508
+2509
+2510
+2511
+2512
+2513
+2514
+2515
+2516
+2517
+2518
+2519
+2520
+2521
+2522
+2523
+2524
+2525
+2526
+2527
+2528
+2529
+2530
+2531
+2532
+2533
+2534
+2535
+2536
+2537
+2538
+2539
+2540
+2541
+2542
+2543
+2544
+2545
+2546
+2547
+2548
+2549
+2550
+2551
+2552
+2553
+2554
+2555
+2556
+2557
+2558
+2559
+2560
+2561
+2562
+2563
+2564
+2565
+2566
+2567
+2568
+2569
+2570
+2571
+2572
+2573
+2574
+2575
+2576
+2577
+2578
+2579
+2580
+2581
+2582
+2583
+2584
+2585
+2586
+2587
+2588
+2589
+2590
+2591
+2592
+2593
+2594
+2595
+2596
+2597
+2598
+2599
+2600
+2601
+2602
+2603
+2604
+2605
+2606
+2607
+2608
+2609
+2610
+2611
+2612
+2613
+2614
+2615
+2616
+2617
+2618
+2619
+2620
+2621
+2622
+2623
+2624
+2625
+2626
+2627
+2628
+2629
+2630
+2631
+2632
+2633
+2634
+2635
+2636
+2637
+2638
+2639
+2640
+2641
+2642
+2643
+2644
+2645
+2646
+2647
+2648
+2649
+2650
+2651
+2652
+2653
+2654
+2655
+2656
+2657
+2658
+2659
+2660
+2661
+2662
+2663
+2664
+2665
+2666
+2667
+2668
+2669
+2670
+2671
+2672
+2673
+2674
+2675
+2676
+2677
+2678
+2679
+2680
+2681
+2682
+2683
+2684
+2685
+2686
+2687
+2688
+2689
+2690
+2691
+2692
+2693
+2694
+2695
+2696
+2697
+2698
+2699
+2700
+2701
+2702
+2703
+2704
+2705
+2706
+2707
+2708
+2709
+2710
+2711
+2712
+2713
+2714
+2715
+2716
+2717
+2718
+2719
+2720
+2721
+2722
+2723
+2724
+2725
+2726
+2727
+2728
+2729
+2730
+2731
+2732
+2733
+2734
+2735
+2736
+2737
+2738
+2739
+2740
+2741
+2742
+2743
+2744
+2745
+2746
+2747
+2748
+2749
+2750
+2751
+2752
+2753
+2754
+2755
+2756
+2757
+2758
+2759
+2760
+2761
+2762
+2763
+2764
+2765
+2766
+2767
+2768
+2769
+2770
+2771
+2772
+2773
+2774
+2775
+2776
+2777
+2778
+2779
+2780
+2781
+2782
+2783
+2784
+2785
+2786
+2787
+2788
+2789
+2790
+2791
+2792
+2793
+2794
+2795
+2796
+2797
+2798
+2799
+2800
+2801
+2802
+2803
+2804
+2805
+2806
+2807
+2808
+2809
+2810
+2811
+2812
+2813
+2814
+2815
+2816
+2817
+2818
+2819
+2820
+2821
+2822
+2823
+2824
+2825
+2826
+2827
+2828
+2829
+2830
+2831
+2832
+2833
+2834
+2835
+2836
+2837
+2838
+2839
+2840
+2841
+2842
+2843
+2844
+2845
+2846
+2847
+2848
+2849
+2850
+2851
+2852
+2853
+2854
+2855
+2856
+2857
+2858
+2859
+2860
+2861
+2862
+2863
+2864
+2865
+2866
+2867
+2868
+2869
+2870
+2871
+2872
+2873
+2874
+2875
+2876
+2877
+2878
+2879
+2880
+2881
+2882
+2883
+2884
+2885
+2886
+2887
+2888
+2889
+2890
+2891
+2892
+2893
+2894
+2895
+2896
+2897
+2898
+2899
+2900
+2901
+2902
+2903
+2904
+2905
+2906
+2907
+2908
+2909
+2910
+2911
+2912
+2913
+2914
+2915
+2916
+2917
+2918
+2919
+2920
+2921
+2922
+2923
+2924
+2925
+2926
+2927
+2928
+2929
+2930
+2931
+2932
+2933
+2934
+2935
+2936
+2937
+2938
+2939
+2940
+2941
+2942
+2943
+2944
+2945
+2946
+2947
+2948
+2949
+2950
+2951
+2952
+2953
+2954
+2955
+2956
+2957
+2958
+2959
+2960
+2961
+2962
+2963
+2964
+2965
+2966
+2967
+2968
+2969
+2970
+2971
+2972
+2973
+2974
+2975
+2976
+2977
+2978
+2979
+2980
+2981
+2982
+2983
+2984
+2985
+2986
+2987
+2988
+2989
+2990
+2991
+2992
+2993
+2994
+2995
+2996
+2997
+2998
+2999
+3000
+3001
+3002
+3003
+3004
+3005
+3006
+3007
+3008
+3009
+3010
+3011
+3012
+3013
+3014
+3015
+3016
+3017
+3018
+3019
+3020
+3021
+3022
+3023
+3024
+3025
+3026
+3027
+3028
+3029
+3030
+3031
+3032
+3033
+3034
+3035
+3036
+3037
+3038
+3039
+3040
+3041
+3042
+3043
+3044
+3045
+3046
+3047
+3048
+3049
+3050
+3051
+3052
+3053
+3054
+3055
+3056
+3057
+3058
+3059
+3060
+3061
+3062
+3063
+3064
+3065
+3066
+3067
+3068
+3069
+3070
+3071
+3072
+3073
+3074
+3075
+3076
+3077
+3078
+3079
+3080
+3081
+3082
+3083
+3084
+3085
+3086
+3087
+3088
+3089
+3090
+3091
+3092
+3093
+3094
+3095
+3096
+3097
+3098
+3099
+3100
+3101
+3102
+3103
+3104
+3105
+3106
+3107
+3108
+3109
+3110
+3111
+3112
+3113
+3114
+3115
+3116
+3117
+3118
+3119
+3120
+3121
+3122
+3123
+3124
+3125
+3126
+3127
+3128
+3129
+3130
+3131
+3132
+3133
+3134
+3135
+3136
+3137
+3138
+3139
+3140
+3141
+3142
+3143
+3144
+3145
+3146
+3147
+3148
+3149
+3150
+3151
+3152
+3153
+3154
+3155
+3156
+3157
+3158
+3159
+3160
+3161
+3162
+3163
+3164
+3165
+3166
+3167
+3168
+3169
+3170
+3171
+3172
+3173
+3174
+3175
+3176
+3177
+3178
+3179
+3180
+3181
+3182
+3183
+3184
+3185
+3186
+3187
+3188
+3189
+3190
+3191
+3192
+3193
+3194
+3195
+3196
+3197
+3198
+3199
+3200
+3201
+3202
+3203
+3204
+3205
+3206
+3207
+3208
+3209
+3210
+3211
+3212
+3213
+3214
+3215
+3216
+3217
+3218
+3219
+3220
+3221
+3222
+3223
+3224
+3225
+3226
+3227
+3228
+3229
+3230
+3231
+3232
+3233
+3234
+3235
+3236
+3237
+3238
+3239
+3240
+3241
+3242
+3243
+3244
+3245
+3246
+3247
+3248
+3249
+3250
+3251
+3252
+3253
+3254
+3255
+3256
+3257
+3258
+3259
+3260
+3261
+3262
+3263
+3264
+3265
+3266
+3267
+3268
+3269
+3270
+3271
+3272
+3273
+3274
+3275
+3276
+3277
+3278
+3279
+3280
+3281
+3282
+3283
+3284
+3285
+3286
+3287
+3288
+3289
+3290
+3291
+3292
+3293
+3294
+3295
+3296
+3297
+3298
+3299
+3300
+3301
+3302
+3303
+3304
+3305
+3306
+3307
+3308
+3309
+3310
+3311
+3312
+3313
+3314
+3315
+3316
+3317
+3318
+3319
+3320
+3321
+3322
+3323
+3324
+3325
+3326
+3327
+3328
+3329
+3330
+3331
+3332
+3333
+3334
+3335
+3336
+3337
+3338
+3339
+3340
+3341
+3342
+3343
+3344
+3345
+3346
+3347
+3348
+3349
+3350
+3351
+3352
+3353
+3354
+3355
+3356
+3357
+3358
+3359
+3360
+3361
+3362
+3363
+3364
+3365
+3366
+3367
+3368
+3369
+3370
+3371
+3372
+3373
+3374
+3375
+3376
+3377
+3378
+3379
+3380
+3381
+3382
+3383
+3384
+3385
+3386
+3387
+3388
+3389
+3390
+3391
+3392
+3393
+3394
+3395
+3396
+3397
+3398
+3399
+3400
+3401
+3402
+3403
+3404
+3405
+3406
+3407
+3408
+3409
+3410
+3411
+3412
+3413
+3414
+3415
+3416
+3417
+3418
+3419
+3420
+3421
+3422
+3423
+3424
+3425
+3426
+3427
+3428
+3429
+3430
+3431
+3432
+3433
+3434
+3435
+3436
+3437
+3438
+3439
+3440
+3441
+3442
+3443
+3444
+3445
+3446
+3447
+3448
+3449
+3450
+3451
+3452
+3453
+3454
+3455
+3456
+3457
+3458
+3459
+3460
+3461
+3462
+3463
+3464
+3465
+3466
+3467
+3468
+3469
+3470
+3471
+3472
+3473
+3474
+3475
+3476
+3477
+3478
+3479
+3480
+3481
+3482
+3483
+3484
+3485
+3486
+3487
+3488
+3489
+3490
+3491
+3492
+3493
+3494
+3495
+3496
+3497
+3498
+3499
+3500
+3501
+3502
+3503
+3504
+3505
+3506
+3507
+3508
+3509
+3510
+3511
+3512
+3513
+3514
+3515
+3516
+3517
+3518
+3519
+3520
+3521
+3522
+3523
+3524
+3525
+3526
+3527
+3528
+3529
+3530
+3531
+3532
+3533
+3534
+3535
+3536
+3537
+3538
+3539
+3540
+3541
+3542
+3543
+3544
+3545
+3546
+3547
+3548
+3549
+3550
+3551
+3552
+3553
+3554
+3555
+3556
+3557
+3558
+3559
+3560
+3561
+3562
+3563
+3564
+3565
+3566
+3567
+3568
+3569
+3570
+3571
+3572
+3573
+3574
+3575
+3576
+3577
+3578
+3579
+3580
+3581
+3582
+3583
+3584
+3585
+3586
+3587
+3588
+3589
+3590
+3591
+3592
+3593
+3594
+3595
+3596
+3597
+3598
+3599
+3600
+3601
+3602
+3603
+3604
+3605
+3606
+3607
+3608
+3609
+3610
+3611
+3612
+3613
+3614
+3615
+3616
+3617
+3618
+3619
+3620
+3621
+3622
+3623
+3624
+3625
+3626
+3627
+3628
+3629
+3630
+3631
+3632
+3633
+3634
+3635
+3636
+3637
+3638
+3639
+3640
+3641
+3642
+3643
+3644
+3645
+3646
+3647
+3648
+3649
+3650
+3651
+3652
+3653
+3654
+3655
+3656
+3657
+3658
+3659
+3660
+3661
+3662
+3663
+3664
+3665
+3666
+3667
+3668
+3669
+3670
+3671
+3672
+3673
+3674
+3675
+3676
+3677
+3678
+3679
+3680
+3681
+3682
+3683
+3684
+3685
+3686
+3687
+3688
+3689
+3690
+3691
+3692
+3693
+3694
+3695
+3696
+3697
+3698
+3699
+3700
+3701
+3702
+3703
+3704
+3705
+3706
+3707
+3708
+3709
+3710
+3711
+3712
+3713
+3714
+3715
+3716
+3717
+3718
+3719
+3720
+3721
+3722
+3723
+3724
+3725
+3726
+3727
+3728
+3729
+3730
+3731
+3732
+3733
+3734
+3735
+3736
+3737
+3738
+3739
+3740
+3741
+3742
+3743
+3744
+3745
+3746
+3747
+3748
+3749
+3750
+3751
+3752
+3753
+3754
+3755
+3756
+3757
+3758
+3759
+3760
+3761
+3762
+3763
+3764
+3765
+3766
+3767
+3768
+3769
+3770
+3771
+3772
+3773
+3774
+3775
+3776
+3777
+3778
+3779
+3780
+3781
+3782
+3783
+3784
+3785
+3786
+3787
+3788
+3789
+3790
+3791
+3792
+3793
+3794
+3795
+3796
+3797
+3798
+3799
+3800
+3801
+3802
+3803
+3804
+3805
+3806
+3807
+3808
+3809
+3810
+3811
+3812
+3813
+3814
+3815
+3816
+3817
+3818
+3819
+3820
+3821
+3822
+3823
+3824
+3825
+3826
+3827
+3828
+3829
+3830
+3831
+3832
+3833
+3834
+3835
+3836
+3837
+3838
+3839
+3840
+3841
+3842
+3843
+3844
+3845
+3846
+3847
+3848
+3849
+3850
+3851
+3852
+3853
+3854
+3855
+3856
+3857
+3858
+3859
+3860
+3861
+3862
+3863
+3864
+3865
+3866
+3867
+3868
+3869
+3870
+3871
+3872
+3873
+3874
+3875
+3876
+3877
+3878
+3879
+3880
+3881
+3882
+3883
+3884
+3885
+3886
+3887
+3888
+3889
+3890
+3891
+3892
+3893
+3894
+3895
+3896
+3897
+3898
+3899
+3900
+3901
+3902
+3903
+3904
+3905
+3906
+3907
+3908
+3909
+3910
+3911
+3912
+3913
+3914
+3915
+3916
+3917
+3918
+3919
+3920
+3921
+3922
+3923
+3924
+3925
+3926
+3927
+3928
+3929
+3930
+3931
+3932
+3933
+3934
+3935
+3936
+3937
+3938
+3939
+3940
+3941
+3942
+3943
+3944
+3945
+3946
+3947
+3948
+3949
+3950
+3951
+3952
+3953
+3954
+3955
+3956
+3957
+3958
+3959
+3960
+3961
+3962
+3963
+3964
+3965
+3966
+3967
+3968
+3969
+3970
+3971
+3972
+3973
+3974
+3975
+3976
+3977
+3978
+3979
+3980
+3981
+3982
+3983
+3984
+3985
+3986
+3987
+3988
+3989
+3990
+3991
+3992
+3993
+3994
+3995
+3996
+3997
+3998
+3999
+4000
+4001
+4002
+4003
+4004
+4005
+4006
+4007
+4008
+4009
+4010
+4011
+4012
+4013
+4014
+4015
+4016
+4017
+4018
+4019
+4020
+4021
+4022
+4023
+4024
+4025
+4026
+4027
+4028
+4029
+4030
+4031
+4032
+4033
+4034
+4035
+4036
+4037
+4038
+4039
+4040
+4041
+4042
+4043
+4044
+4045
+4046
+4047
+4048
+4049
+4050
+4051
+4052
+4053
+4054
+4055
+4056
+4057
+4058
+4059
+4060
+4061
+4062
+4063
+4064
+4065
+4066
+4067
+4068
+4069
+4070
+4071
+4072
+4073
+4074
+4075
+4076
+4077
+4078
+4079
+4080
+4081
+4082
+4083
+4084
+4085
+4086
+4087
+4088
+4089
+4090
+4091
+4092
+4093
+4094
+4095
+4096
+4097
+4098
+4099
+4100
+4101
+4102
+4103
+4104
+4105
+4106
+4107
+4108
+4109
+4110
+4111
+4112
+4113
+4114
+4115
+4116
+4117
+4118
+4119
+4120
+4121
+4122
+4123
+4124
+4125
+4126
+4127
+4128
+4129
+4130
+4131
+4132
+4133
+4134
+4135
+4136
+4137
+4138
+4139
+4140
+4141
+4142
+4143
+4144
+4145
+4146
+4147
+4148
+4149
+4150
+4151
+4152
+4153
+4154
+4155
+4156
+4157
+4158
+4159
+4160
+4161
+4162
+4163
+4164
+4165
+4166
+4167
+4168
+4169
+4170
+4171
+4172
+4173
+4174
+4175
+4176
+4177
+4178
+4179
+4180
+4181
+4182
+4183
+4184
+4185
+4186
+4187
+4188
+4189
+4190
+4191
+4192
+4193
+4194
+4195
+4196
+4197
+4198
+4199
+4200
+4201
+4202
+4203
+4204
+4205
+4206
+4207
+4208
+4209
+4210
+4211
+4212
+4213
+4214
+4215
+4216
+4217
+4218
+4219
+4220
+4221
+4222
+4223
+4224
+4225
+4226
+4227
+4228
+4229
+4230
+4231
+4232
+4233
+4234
+4235
+4236
+4237
+4238
+4239
+4240
+4241
+4242
+4243
+4244
+4245
+4246
+4247
+4248
+4249
+4250
+4251
+4252
+4253
+4254
+4255
+4256
+4257
+4258
+4259
+4260
+4261
+4262
+4263
+4264
+4265
+4266
+4267
+4268
+4269
+4270
+4271
+4272
+4273
+4274
+4275
+4276
+4277
+4278
+4279
+4280
+4281
+4282
+4283
+4284
+4285
+4286
+4287
+4288
+4289
+4290
+4291
+4292
+4293
+4294
+4295
+4296
+4297
+4298
+4299
+4300
+4301
+4302
+4303
+4304
+4305
+4306
+4307
+4308
+4309
+4310
+4311
+4312
+4313
+4314
+4315
+4316
+4317
+4318
+4319
+4320
+4321
+4322
+4323
+4324
+4325
+4326
+4327
+4328
+4329
+4330
+4331
+4332
+4333
+4334
+4335
+4336
+4337
+4338
+4339
+4340
+4341
+4342
+4343
+4344
+4345
+4346
+4347
+4348
+4349
+4350
+4351
+4352
+4353
+4354
+4355
+4356
+4357
+4358
+4359
+4360
+4361
+4362
+4363
+4364
+4365
+4366
+4367
+4368
+4369
+4370
+4371
+4372
+4373
+4374
+4375
+4376
+4377
+4378
+4379
+4380
+4381
+4382
+4383
+4384
+4385
+4386
+4387
+4388
+4389
+4390
+4391
+4392
+4393
+4394
+4395
+4396
+4397
+4398
+4399
+4400
+4401
+4402
+4403
+4404
+4405
+4406
+4407
+4408
+4409
+4410
+4411
+4412
+4413
+4414
+4415
+4416
+4417
+4418
+4419
+4420
+4421
+4422
+4423
+4424
+4425
+4426
+4427
+4428
+4429
+4430
+4431
+4432
+4433
+4434
+4435
+4436
+4437
+4438
+4439
+4440
+4441
+4442
+4443
+4444
+4445
+4446
+4447
+4448
+4449
+4450
+4451
+4452
+4453
+4454
+4455
+4456
+4457
+4458
+4459
+4460
+4461
+4462
+4463
+4464
+4465
+4466
+4467
+4468
+4469
+4470
+4471
+4472
+4473
+4474
+4475
+4476
+4477
+4478
+4479
+4480
+4481
+4482
+4483
+4484
+4485
+4486
+4487
+4488
+4489
+4490
+4491
+4492
+4493
+4494
+4495
+4496
+4497
+4498
+4499
+4500
+4501
+4502
+4503
+4504
+4505
+4506
+4507
+4508
+4509
+4510
+4511
+4512
+4513
+4514
+4515
+4516
+4517
+4518
+4519
+4520
+4521
+4522
+4523
+4524
+4525
+4526
+4527
+4528
+4529
+4530
+4531
+4532
+4533
+4534
+4535
+4536
+4537
+4538
+4539
+4540
+4541
+4542
+4543
+4544
+4545
+4546
+4547
+4548
+4549
+4550
+4551
+4552
+4553
+4554
+4555
+4556
+4557
+4558
+4559
+4560
+4561
+4562
+4563
+4564
+4565
+4566
+4567
+4568
+4569
+4570
+4571
+4572
+4573
+4574
+4575
+4576
+4577
+4578
+4579
+4580
+4581
+4582
+4583
+4584
+4585
+4586
+4587
+4588
+4589
+4590
+4591
+4592
+4593
+4594
+4595
+4596
+4597
+4598
+4599
+4600
+4601
+4602
+4603
+4604
+4605
+4606
+4607
+4608
+4609
+4610
+4611
+4612
+4613
+4614
+4615
+4616
+4617
+4618
+4619
+4620
+4621
+4622
+4623
+4624
+4625
+4626
+4627
+4628
+4629
+4630
+4631
+4632
+4633
+4634
+4635
+4636
+4637
+4638
+4639
+4640
+4641
+4642
+4643
+4644
+4645
+4646
+4647
+4648
+4649
+4650
+4651
+4652
+4653
+4654
+4655
+4656
+4657
+4658
+4659
+4660
+4661
+4662
+4663
+4664
+4665
+4666
+4667
+4668
+4669
+4670
+4671
+4672
+4673
+4674
+4675
+4676
+4677
+4678
+4679
+4680
+4681
+4682
+4683
+4684
+4685
+4686
+4687
+4688
+4689
+4690
+4691
+4692
+4693
+4694
+4695
+4696
+4697
+4698
+4699
+4700
+4701
+4702
+4703
+4704
+4705
+4706
+4707
+4708
+4709
+4710
+4711
+4712
+4713
+4714
+4715
+4716
+4717
+4718
+4719
+4720
+4721
+4722
+4723
+4724
+4725
+4726
+4727
+4728
+4729
+4730
+4731
+4732
+4733
+4734
+4735
+4736
+4737
+4738
+4739
+4740
+4741
+4742
+4743
+4744
+4745
+4746
+4747
+4748
+4749
+4750
+4751
+4752
+4753
+4754
+4755
+4756
+4757
+4758
+4759
+4760
+4761
+4762
+4763
+4764
+4765
+4766
+4767
+4768
+4769
+4770
+4771
+4772
+4773
+4774
+4775
+4776
+4777
+4778
+4779
+4780
+4781
+4782
+4783
+4784
+4785
+4786
+4787
+4788
+4789
+4790
+4791
+4792
+4793
+4794
+4795
+4796
+4797
+4798
+4799
+4800
+4801
+4802
+4803
+4804
+4805
+4806
+4807
+4808
+4809
+4810
+4811
+4812
+4813
+4814
+4815
+4816
+4817
+4818
+4819
+4820
+4821
+4822
+4823
+4824
+4825
+4826
+4827
+4828
+4829
+4830
+4831
+4832
+4833
+4834
+4835
+4836
+4837
+4838
+4839
+4840
+4841
+4842
+4843
+4844
+4845
+4846
+4847
+4848
+4849
+4850
+4851
+4852
+4853
+4854
+4855
+4856
+4857
+4858
+4859
+4860
+4861
+4862
+4863
+4864
+4865
+4866
+4867
+4868
+4869
+4870
+4871
+4872
+4873
+4874
+4875
+4876
+4877
+4878
+4879
+4880
+4881
+4882
+4883
+4884
+4885
+4886
+4887
+4888
+4889
+4890
+4891
+4892
+4893
+4894
+4895
+4896
+4897
+4898
+4899
+4900
+4901
+4902
+4903
+4904
+4905
+4906
+4907
+4908
+4909
+4910
+4911
+4912
+4913
+4914
+4915
+4916
+4917
+4918
+4919
+4920
+4921
+4922
+4923
+4924
+4925
+4926
+4927
+4928
+4929
+4930
+4931
+4932
+4933
+4934
+4935
+4936
+4937
+4938
+4939
+4940
+4941
+4942
+4943
+4944
+4945
+4946
+4947
+4948
+4949
+4950
+4951
+4952
+4953
+4954
+4955
+4956
+4957
+4958
+4959
+4960
+4961
+4962
+4963
+4964
+4965
+4966
+4967
+4968
+4969
+4970
+4971
+4972
+4973
+4974
+4975
+4976
+4977
+4978
+4979
+4980
+4981
+4982
+4983
+4984
+4985
+4986
+4987
+4988
+4989
+4990
+4991
+4992
+4993
+4994
+4995
+4996
+4997
+4998
+4999
+5000
+5001
+5002
+5003
+5004
+5005
+5006
+5007
+5008
+5009
+5010
+5011
+5012
+5013
+5014
+5015
+5016
+5017
+5018
+5019
+5020
+5021
+5022
+5023
+5024
+5025
+5026
+5027
+5028
+5029
+5030
+5031
+5032
+5033
+5034
+5035
+5036
+5037
+5038
+5039
+5040
+5041
+5042
+5043
+5044
+5045
+5046
+5047
+5048
+5049
+5050
+5051
+5052
+5053
+5054
+5055
+5056
+5057
+5058
+5059
+5060
+5061
+5062
+5063
+5064
+5065
+5066
+5067
+5068
+5069
+5070
+5071
+5072
+5073
+5074
+5075
+5076
+5077
+5078
+5079
+5080
+5081
+5082
+5083
+5084
+5085
+5086
+5087
+5088
+5089
+5090
+5091
+5092
+5093
+5094
+5095
+5096
+5097
+5098
+5099
+5100
+5101
+5102
+5103
+5104
+5105
+5106
+5107
+5108
+5109
+5110
+5111
+5112
+5113
+5114
+5115
+5116
+5117
+5118
+5119
+5120
+5121
+5122
+5123
+5124
+5125
+5126
+5127
+5128
+5129
+5130
+5131
+5132
+5133
+5134
+5135
+5136
+5137
+5138
+5139
+5140
+5141
+5142
+5143
+5144
+5145
+5146
+5147
+5148
+5149
+5150
+5151
+5152
+5153
+5154
+5155
+5156
+5157
+5158
+5159
+5160
+5161
+5162
+5163
+5164
+5165
+5166
+5167
+5168
+5169
+5170
+5171
+5172
+5173
+5174
+5175
+5176
+5177
+5178
+5179
+5180
+5181
+5182
+5183
+5184
+5185
+5186
+5187
+5188
+5189
+5190
+5191
+5192
+5193
+5194
+5195
+5196
+5197
+5198
+5199
+5200
+5201
+5202
+5203
+5204
+5205
+5206
+5207
+5208
+5209
+5210
+5211
+5212
+5213
+5214
+5215
+5216
+5217
+5218
+5219
+5220
+5221
+5222
+5223
+5224
+5225
+5226
+5227
+5228
+5229
+5230
+5231
+5232
+5233
+5234
+5235
+5236
+5237
+5238
+5239
+5240
+5241
+5242
+5243
+5244
+5245
+5246
+5247
+5248
+5249
+5250
+5251
+5252
+5253
+5254
+5255
+5256
+5257
+5258
+5259
+5260
+5261
+5262
+5263
+5264
+5265
+5266
+5267
+5268
+5269
+5270
+5271
+5272
+5273
+5274
+5275
+5276
+5277
+5278
+5279
+5280
+5281
+5282
+5283
+5284
+5285
+5286
+5287
+5288
+5289
+5290
+5291
+5292
+5293
+5294
+5295
+5296
+5297
+5298
+5299
+5300
+5301
+5302
+5303
+5304
+5305
+5306
+5307
+5308
+5309
+5310
+5311
+5312
+5313
+5314
+5315
+5316
+5317
+5318
+5319
+5320
+5321
+5322
+5323
+5324
+5325
+5326
+5327
+5328
+5329
+5330
+5331
+5332
+5333
+5334
+5335
+5336
+5337
+5338
+5339
+5340
+5341
+5342
+5343
+5344
+5345
+5346
+5347
+5348
+5349
+5350
+5351
+5352
+5353
+5354
+5355
+5356
+5357
+5358
+5359
+5360
+5361
+5362
+5363
+5364
+5365
+5366
+5367
+5368
+5369
+5370
+5371
+5372
+5373
+5374
+5375
+5376
+5377
+5378
+5379
+5380
+5381
+5382
+5383
+5384
+5385
+5386
+5387
+5388
+5389
+5390
+5391
+5392
+5393
+5394
+5395
+5396
+5397
+5398
+5399
+5400
+5401
+5402
+5403
+5404
+5405
+5406
+5407
+5408
+5409
+5410
+5411
+5412
+5413
+5414
+5415
+5416
+5417
+5418
+5419
+5420
+5421
+5422
+5423
+5424
+5425
+5426
+5427
+5428
+5429
+5430
+5431
+5432
+5433
+5434
+5435
+5436
+5437
+5438
+5439
+5440
+5441
+5442
+5443
+5444
+5445
+5446
+5447
+5448
+5449
+5450
+5451
+5452
+5453
+5454
+5455
+5456
+5457
+5458
+5459
+5460
+5461
+5462
+5463
+5464
+5465
+5466
+5467
+5468
+5469
+5470
+5471
+5472
+5473
+5474
+5475
+5476
+5477
+5478
+5479
+5480
+5481
+5482
+5483
+5484
+5485
+5486
+5487
+5488
+5489
+5490
+5491
+5492
+5493
+5494
+5495
+5496
+5497
+5498
+5499
+5500
+5501
+5502
+5503
+5504
+5505
+5506
+5507
+5508
+5509
+5510
+5511
+5512
+5513
+5514
+5515
+5516
+5517
+5518
+5519
+5520
+5521
+5522
+5523
+5524
+5525
+5526
+5527
+5528
+5529
+5530
+5531
+5532
+5533
+5534
+5535
+5536
+5537
+5538
+5539
+5540
+5541
+5542
+5543
+5544
+5545
+5546
+5547
+5548
+5549
+5550
+5551
+5552
+5553
+5554
+5555
+5556
+5557
+5558
+5559
+5560
+5561
+5562
+5563
+5564
+5565
+5566
+5567
+5568
+5569
+5570
+5571
+5572
+5573
+5574
+5575
+5576
+5577
+5578
+5579
+5580
+5581
+5582
+5583
+5584
+5585
+5586
+5587
+5588
+5589
+5590
+5591
+5592
+5593
+5594
+5595
+5596
+5597
+5598
+5599
+5600
+5601
+5602
+5603
+5604
+5605
+5606
+5607
+5608
+5609
+5610
+5611
+5612
+5613
+5614
+5615
+5616
+5617
+5618
+5619
+5620
+5621
+5622
+5623
+5624
+5625
+5626
+5627
+5628
+5629
+5630
+5631
+5632
+5633
+5634
+5635
+5636
+5637
+5638
+5639
+5640
+5641
+5642
+5643
+5644
+5645
+5646
+5647
+5648
+5649
+5650
+5651
+5652
+5653
+5654
+5655
+5656
+5657
+5658
+5659
+5660
+5661
+5662
+5663
+5664
+5665
+5666
+5667
+5668
+5669
+5670
+5671
+5672
+5673
+5674
+5675
+5676
+5677
+5678
+5679
+5680
+5681
+5682
+5683
+5684
+5685
+5686
+5687
+5688
+5689
+5690
+5691
+5692
+5693
+5694
+5695
+5696
+5697
+5698
+5699
+5700
+5701
+5702
+5703
+5704
+5705
+5706
+5707
+5708
+5709
+5710
+5711
+5712
+5713
+5714
+5715
+5716
+5717
+5718
+5719
+5720
+5721
+5722
+5723
+5724
+5725
+5726
+5727
+5728
+5729
+5730
+5731
+5732
+5733
+5734
+5735
+5736
+5737
+5738
+5739
+5740
+5741
+5742
+5743
+5744
+5745
+5746
+5747
+5748
+5749
+5750
+5751
+5752
+5753
+5754
+5755
+5756
+5757
+5758
+5759
+5760
+5761
+5762
+5763
+5764
+5765
+5766
+5767
+5768
+5769
+5770
+5771
+5772
+5773
+5774
+5775
+5776
+5777
+5778
+5779
+5780
+5781
+5782
+5783
+5784
+5785
+5786
+5787
+5788
+5789
+5790
+5791
+5792
+5793
+5794
+5795
+5796
+5797
+5798
+5799
+5800
+5801
+5802
+5803
+5804
+5805
+5806
+5807
+5808
+5809
+5810
+5811
+5812
+5813
+5814
+5815
+5816
+5817
+5818
+5819
+5820
+5821
+5822
+5823
+5824
+5825
+5826
+5827
+5828
+5829
+5830
+5831
+5832
+5833
+5834
+5835
+5836
+5837
+5838
+5839
+5840
+5841
+5842
+5843
+5844
+5845
+5846
+5847
+5848
+5849
+5850
+5851
+5852
+5853
+5854
+5855
+5856
+5857
+5858
+5859
+5860
+5861
+5862
+5863
+5864
+5865
+5866
+5867
+5868
+5869
+5870
+5871
+5872
+5873
+5874
+5875
+5876
+5877
+5878
+5879
+5880
+5881
+5882
+5883
+5884
+5885
+5886
+5887
+5888
+5889
+5890
+5891
+5892
+5893
+5894
+5895
+5896
+5897
+5898
+5899
+5900
+5901
+5902
+5903
+5904
+5905
+5906
+5907
+5908
+5909
+5910
+5911
+5912
+5913
+5914
+5915
+5916
+5917
+5918
+5919
+5920
+5921
+5922
+5923
+5924
+5925
+5926
+5927
+5928
+5929
+5930
+5931
+5932
+5933
+5934
+5935
+5936
+5937
+5938
+5939
+5940
+5941
+5942
+5943
+5944
+5945
+5946
+5947
+5948
+5949
+5950
+5951
+5952
+5953
+5954
+5955
+5956
+5957
+5958
+5959
+5960
+5961
+5962
+5963
+5964
+5965
+5966
+5967
+5968
+5969
+5970
+5971
+5972
+5973
+5974
+5975
+5976
+5977
+5978
+5979
+5980
+5981
+5982
+5983
+5984
+5985
+5986
+5987
+5988
+5989
+5990
+5991
+5992
+5993
+5994
+5995
+5996
+5997
+5998
+5999
+6000
+6001
+6002
+6003
+6004
+6005
+6006
+6007
+6008
+6009
+6010
+6011
+6012
+6013
+6014
+6015
+6016
+6017
+6018
+6019
+6020
+6021
+6022
+6023
+6024
+6025
+6026
+6027
+6028
+6029
+6030
+6031
+6032
+6033
+6034
+6035
+6036
+6037
+6038
+6039
+6040
+6041
+6042
+6043
+6044
+6045
+6046
+6047
+6048
+6049
+6050
+6051
+6052
+6053
+6054
+6055
+6056
+6057
+6058
+6059
+6060
+6061
+6062
+6063
+6064
+6065
+6066
+6067
+6068
+6069
+6070
+6071
+6072
+6073
+6074
+6075
+6076
+6077
+6078
+6079
+6080
+6081
+6082
+6083
+6084
+6085
+6086
+6087
+6088
+6089
+6090
+6091
+6092
+6093
+6094
+6095
+6096
+6097
+6098
+6099
+6100
+6101
+6102
+6103
+6104
+6105
+6106
+6107
+6108
+6109
+6110
+6111
+6112
+6113
+6114
+6115
+6116
+6117
+6118
+6119
+6120
+6121
+6122
+6123
+6124
+6125
+6126
+6127
+6128
+6129
+6130
+6131
+6132
+6133
+6134
+6135
+6136
+6137
+6138
+6139
+6140
+6141
+6142
+6143
+6144
+6145
+6146
+6147
+6148
+6149
+6150
+6151
+6152
+6153
+6154
+6155
+6156
+6157
+6158
+6159
+6160
+6161
+6162
+6163
+6164
+6165
+6166
+6167
+6168
+6169
+6170
+6171
+6172
+6173
+6174
+6175
+6176
+6177
+6178
+6179
+6180
+6181
+6182
+6183
+6184
+6185
+6186
+6187
+6188
+6189
+6190
+6191
+6192
+6193
+6194
+6195
+6196
+6197
+6198
+6199
+6200
+6201
+6202
+6203
+6204
+6205
+6206
+6207
+6208
+6209
+6210
+6211
+6212
+6213
+6214
+6215
+6216
+6217
+6218
+6219
+6220
+6221
+6222
+6223
+6224
+6225
+6226
+6227
+6228
+6229
+6230
+6231
+6232
+6233
+6234
+6235
+6236
+6237
+6238
+6239
+6240
+6241
+6242
+6243
+6244
+6245
+6246
+6247
+6248
+6249
+6250
+6251
+6252
+6253
+6254
+6255
+6256
+6257
+6258
+6259
+6260
+6261
+6262
+6263
+6264
+6265
+6266
+6267
+6268
+6269
+6270
+6271
+6272
+6273
+6274
+6275
+6276
+6277
+6278
+6279
+6280
+6281
+6282
+6283
+6284
+6285
+6286
+6287
+6288
+6289
+6290
+6291
+6292
+6293
+6294
+6295
+6296
+6297
+6298
+6299
+6300
+6301
+6302
+6303
+6304
+6305
+6306
+6307
+6308
+6309
+6310
+6311
+6312
+6313
+6314
+6315
+6316
+6317
+6318
+6319
+6320
+6321
+6322
+6323
+6324
+6325
+6326
+6327
+6328
+6329
+6330
+6331
+6332
+6333
+6334
+6335
+6336
+6337
+6338
+6339
+6340
+6341
+6342
+6343
+6344
+6345
+6346
+6347
+6348
+6349
+6350
+6351
+6352
+6353
+6354
+6355
+6356
+6357
+6358
+6359
+6360
+6361
+6362
+6363
+6364
+6365
+6366
+6367
+6368
+6369
+6370
+6371
+6372
+6373
+6374
+6375
+6376
+6377
+6378
+6379
+6380
+6381
+6382
+6383
+6384
+6385
+6386
+6387
+6388
+6389
+6390
+6391
+6392
+6393
+6394
+6395
+6396
+6397
+6398
+6399
+6400
+6401
+6402
+6403
+6404
+6405
+6406
+6407
+6408
+6409
+6410
+6411
+6412
+6413
+6414
+6415
+6416
+6417
+6418
+6419
+6420
+6421
+6422
+6423
+6424
+6425
+6426
+6427
+6428
+6429
+6430
+6431
+6432
+6433
+6434
+6435
+6436
+6437
+6438
+6439
+6440
+6441
+6442
+6443
+6444
+6445
+6446
+6447
+6448
+6449
+6450
+6451
+6452
+6453
+6454
+6455
+6456
+6457
+6458
+6459
+6460
+6461
+6462
+6463
+6464
+6465
+6466
+6467
+6468
+6469
+6470
+6471
+6472
+6473
+6474
+6475
+6476
+6477
+6478
+6479
+6480
+6481
+6482
+6483
+6484
+6485
+6486
+6487
+6488
+6489
+6490
+6491
+6492
+6493
+6494
+6495
+6496
+6497
+6498
+6499
+6500
+6501
+6502
+6503
+6504
+6505
+6506
+6507
+6508
+6509
+6510
+6511
+6512
+6513
+6514
+6515
+6516
+6517
+6518
+6519
+6520
+6521
+6522
+6523
+6524
+6525
+6526
+6527
+6528
+6529
+6530
+6531
+6532
+6533
+6534
+6535
+6536
+6537
+6538
+6539
+6540
+6541
+6542
+6543
+6544
+6545
+6546
+6547
+6548
+6549
+6550
+6551
+6552
+6553
+6554
+6555
+6556
+6557
+6558
+6559
+6560
+6561
+6562
+6563
+6564
+6565
+6566
+6567
+6568
+6569
+6570
+6571
+6572
+6573
+6574
+6575
+6576
+6577
+6578
+6579
+6580
+6581
+6582
+6583
+6584
+6585
+6586
+6587
+6588
+6589
+6590
+6591
+6592
+6593
+6594
+6595
+6596
+6597
+6598
+6599
+6600
+6601
+6602
+6603
+6604
+6605
+6606
+6607
+6608
+6609
+6610
+6611
+6612
+6613
+6614
+6615
+6616
+6617
+6618
+6619
+6620
+6621
+6622
+6623
+6624
+6625
+6626
+6627
+6628
+6629
+6630
+6631
+6632
+6633
+6634
+6635
+6636
+6637
+6638
+6639
+6640
+6641
+6642
+6643
+6644
+6645
+6646
+6647
+6648
+6649
+6650
+6651
+6652
+6653
+6654
+6655
+6656
+6657
+6658
+6659
+6660
+6661
+6662
+6663
+6664
+6665
+6666
+6667
+6668
+6669
+6670
+6671
+6672
+6673
+6674
+6675
+6676
+6677
+6678
+6679
+6680
+6681
+6682
+6683
+6684
+6685
+6686
+6687
+6688
+6689
+6690
+6691
+6692
+6693
+6694
+6695
+6696
+6697
+6698
+6699
+6700
+6701
+6702
+6703
+6704
+6705
+6706
+6707
+6708
+6709
+6710
+6711
+6712
+6713
+6714
+6715
+6716
+6717
+6718
+6719
+6720
+6721
+6722
+6723
+6724
+6725
+6726
+6727
+6728
+6729
+6730
+6731
+6732
+6733
+6734
+6735
+6736
+6737
+6738
+6739
+6740
+6741
+6742
+6743
+6744
+6745
+6746
+6747
+6748
+6749
+6750
+6751
+6752
+6753
+6754
+6755
+6756
+6757
+6758
+6759
+6760
+6761
+6762
+6763
+6764
+6765
+6766
+6767
+6768
+6769
+6770
+6771
+6772
+6773
+6774
+6775
+6776
+6777
+6778
+6779
+6780
+6781
+6782
+6783
+6784
+6785
+6786
+6787
+6788
+6789
+6790
+6791
+6792
+6793
+6794
+6795
+6796
+6797
+6798
+6799
+6800
+6801
+6802
+6803
+6804
+6805
+6806
+6807
+6808
+6809
+6810
+6811
+6812
+6813
+6814
+6815
+6816
+6817
+6818
+6819
+6820
+6821
+6822
+6823
+6824
+6825
+6826
+6827
+6828
+6829
+6830
+6831
+6832
+6833
+6834
+6835
+6836
+6837
+6838
+6839
+6840
+6841
+6842
+6843
+6844
+6845
+6846
+6847
+6848
+6849
+6850
+6851
+6852
+6853
+6854
+6855
+6856
+6857
+6858
+6859
+6860
+6861
+6862
+6863
+6864
+6865
+6866
+6867
+6868
+6869
+6870
+6871
+6872
+6873
+6874
+6875
+6876
+6877
+6878
+6879
+6880
+6881
+6882
+6883
+6884
+6885
+6886
+6887
+6888
+6889
+6890
+6891
+6892
+6893
+6894
+6895
+6896
+6897
+6898
+6899
+6900
+6901
+6902
+6903
+6904
+6905
+6906
+6907
+6908
+6909
+6910
+6911
+6912
+6913
+6914
+6915
+6916
+6917
+6918
+6919
+6920
+6921
+6922
+6923
+6924
+6925
+6926
+6927
+6928
+6929
+6930
+6931
+6932
+6933
+6934
+6935
+6936
+6937
+6938
+6939
+6940
+6941
+6942
+6943
+6944
+6945
+6946
+6947
+6948
+6949
+6950
+6951
+6952
+6953
+6954
+6955
+6956
+6957
+6958
+6959
+6960
+6961
+6962
+6963
+6964
+6965
+6966
+6967
+6968
+6969
+6970
+6971
+6972
+6973
+6974
+6975
+6976
+6977
+6978
+6979
+6980
+6981
+6982
+6983
+6984
+6985
+6986
+6987
+6988
+6989
+6990
+6991
+6992
+6993
+6994
+6995
+6996
+6997
+6998
+6999
+7000
+7001
+7002
+7003
+7004
+7005
+7006
+7007
+7008
+7009
+7010
+7011
+7012
+7013
+7014
+7015
+7016
+7017
+7018
+7019
+7020
+7021
+7022
+7023
+7024
+7025
+7026
+7027
+7028
+7029
+7030
+7031
+7032
+7033
+7034
+7035
+7036
+7037
+7038
+7039
+7040
+7041
+7042
+7043
+7044
+7045
+7046
+7047
+7048
+7049
+7050
+7051
+7052
+7053
+7054
+7055
+7056
+7057
+7058
+7059
+7060
+7061
+7062
+7063
+7064
+7065
+7066
+7067
+7068
+7069
+7070
+7071
+7072
+7073
+7074
+7075
+7076
+7077
+7078
+7079
+7080
+7081
+7082
+7083
+7084
+7085
+7086
+7087
+7088
+7089
+7090
+7091
+7092
+7093
+7094
+7095
+7096
+7097
+7098
+7099
+7100
+7101
+7102
+7103
+7104
+7105
+7106
+7107
+7108
+7109
+7110
+7111
+7112
+7113
+7114
+7115
+7116
+7117
+7118
+7119
+7120
+7121
+7122
+7123
+7124
+7125
+7126
+7127
+7128
+7129
+7130
+7131
+7132
+7133
+7134
+7135
+7136
+7137
+7138
+7139
+7140
+7141
+7142
+7143
+7144
+7145
+7146
+7147
+7148
+7149
+7150
+7151
+7152
+7153
+7154
+7155
+7156
+7157
+7158
+7159
+7160
+7161
+7162
+7163
+7164
+7165
+7166
+7167
+7168
+7169
+7170
+7171
+7172
+7173
+7174
+7175
+7176
+7177
+7178
+7179
+7180
+7181
+7182
+7183
+7184
+7185
+7186
+7187
+7188
+7189
+7190
+7191
+7192
+7193
+7194
+7195
+7196
+7197
+7198
+7199
+7200
+7201
+7202
+7203
+7204
+7205
+7206
+7207
+7208
+7209
+7210
+7211
+7212
+7213
+7214
+7215
+7216
+7217
+7218
+7219
+7220
+7221
+7222
+7223
+7224
+7225
+7226
+7227
+7228
+7229
+7230
+7231
+7232
+7233
+7234
+7235
+7236
+7237
+7238
+7239
+7240
+7241
+7242
+7243
+7244
+7245
+7246
+7247
+7248
+7249
+7250
+7251
+7252
+7253
+7254
+7255
+7256
+7257
+7258
+7259
+7260
+7261
+7262
+7263
+7264
+7265
+7266
+7267
+7268
+7269
+7270
+7271
+7272
+7273
+7274
+7275
+7276
+7277
+7278
+7279
+7280
+7281
+7282
+7283
+7284
+7285
+7286
+7287
+7288
+7289
+7290
+7291
+7292
+7293
+7294
+7295
+7296
+7297
+7298
+7299
+7300
+7301
+7302
+7303
+7304
+7305
+7306
+7307
+7308
+7309
+7310
+7311
+7312
+7313
+7314
+7315
+7316
+7317
+7318
+7319
+7320
+7321
+7322
+7323
+7324
+7325
+7326
+7327
+7328
+7329
+7330
+7331
+7332
+7333
+7334
+7335
+7336
+7337
+7338
+7339
+7340
+7341
+7342
+7343
+7344
+7345
+7346
+7347
+7348
+7349
+7350
+7351
+7352
+7353
+7354
+7355
+7356
+7357
+7358
+7359
+7360
+7361
+7362
+7363
+7364
+7365
+7366
+7367
+7368
+7369
+7370
+7371
+7372
+7373
+7374
+7375
+7376
+7377
+7378
+7379
+7380
+7381
+7382
+7383
+7384
+7385
+7386
+7387
+7388
+7389
+7390
+7391
+7392
+7393
+7394
+7395
+7396
+7397
+7398
+7399
+7400
+7401
+7402
+7403
+7404
+7405
+7406
+7407
+7408
+7409
+7410
+7411
+7412
+7413
+7414
+7415
+7416
+7417
+7418
+7419
+7420
+7421
+7422
+7423
+7424
+7425
+7426
+7427
+7428
+7429
+7430
+7431
+7432
+7433
+7434
+7435
+7436
+7437
+7438
+7439
+7440
+7441
+7442
+7443
+7444
+7445
+7446
+7447
+7448
+7449
+7450
+7451
+7452
+7453
+7454
+7455
+7456
+7457
+7458
+7459
+7460
+7461
+7462
+7463
+7464
+7465
+7466
+7467
+7468
+7469
+7470
+7471
+7472
+7473
+7474
+7475
+7476
+7477
+7478
+7479
+7480
+7481
+7482
+7483
+7484
+7485
+7486
+7487
+7488
+7489
+7490
+7491
+7492
+7493
+7494
+7495
+7496
+7497
+7498
+7499
+7500
+7501
+7502
+7503
+7504
+7505
+7506
+7507
+7508
+7509
+7510
+7511
+7512
+7513
+7514
+7515
+7516
+7517
+7518
+7519
+7520
+7521
+7522
+7523
+7524
+7525
+7526
+7527
+7528
+7529
+7530
+7531
+7532
+7533
+7534
+7535
+7536
+7537
+7538
+7539
+7540
+7541
+7542
+7543
+7544
+7545
+7546
+7547
+7548
+7549
+7550
+7551
+7552
+7553
+7554
+7555
+7556
+7557
+7558
+7559
+7560
+7561
+7562
+7563
+7564
+7565
+7566
+7567
+7568
+7569
+7570
+7571
+7572
+7573
+7574
+7575
+7576
+7577
+7578
+7579
+7580
+7581
+7582
+7583
+7584
+7585
+7586
+7587
+7588
+7589
+7590
+7591
+7592
+7593
+7594
+7595
+7596
+7597
+7598
+7599
+7600
+7601
+7602
+7603
+7604
+7605
+7606
+7607
+7608
+7609
+7610
+7611
+7612
+7613
+7614
+7615
+7616
+7617
+7618
+7619
+7620
+7621
+7622
+7623
+7624
+7625
+7626
+7627
+7628
+7629
+7630
+7631
+7632
+7633
+7634
+7635
+7636
+7637
+7638
+7639
+7640
+7641
+7642
+7643
+7644
+7645
+7646
+7647
+7648
+7649
+7650
+7651
+7652
+7653
+7654
+7655
+7656
+7657
+7658
+7659
+7660
+7661
+7662
+7663
+7664
+7665
+7666
+7667
+7668
+7669
+7670
+7671
+7672
+7673
+7674
+7675
+7676
+7677
+7678
+7679
+7680
+7681
+7682
+7683
+7684
+7685
+7686
+7687
+7688
+7689
+7690
+7691
+7692
+7693
+7694
+7695
+7696
+7697
+7698
+7699
+7700
+7701
+7702
+7703
+7704
+7705
+7706
+7707
+7708
+7709
+7710
+7711
+7712
+7713
+7714
+7715
+7716
+7717
+7718
+7719
+7720
+7721
+7722
+7723
+7724
+7725
+7726
+7727
+7728
+7729
+7730
+7731
+7732
+7733
+7734
+7735
+7736
+7737
+7738
+7739
+7740
+7741
+7742
+7743
+7744
+7745
+7746
+7747
+7748
+7749
+7750
+7751
+7752
+7753
+7754
+7755
+7756
+7757
+7758
+7759
+7760
+7761
+7762
+7763
+7764
+7765
+7766
+7767
+7768
+7769
+7770
+7771
+7772
+7773
+7774
+7775
+7776
+7777
+7778
+7779
+7780
+7781
+7782
+7783
+7784
+7785
+7786
+7787
+7788
+7789
+7790
+7791
+7792
+7793
+7794
+7795
+7796
+7797
+7798
+7799
+7800
+7801
+7802
+7803
+7804
+7805
+7806
+7807
+7808
+7809
+7810
+7811
+7812
+7813
+7814
+7815
+7816
+7817
+7818
+7819
+7820
+7821
+7822
+7823
+7824
+7825
+7826
+7827
+7828
+7829
+7830
+7831
+7832
+7833
+7834
+7835
+7836
+7837
+7838
+7839
+7840
+7841
+7842
+7843
+7844
+7845
+7846
+7847
+7848
+7849
+7850
+7851
+7852
+7853
+7854
+7855
+7856
+7857
+7858
+7859
+7860
+7861
+7862
+7863
+7864
+7865
+7866
+7867
+7868
+7869
+7870
+7871
+7872
+7873
+7874
+7875
+7876
+7877
+7878
+7879
+7880
+7881
+7882
+7883
+7884
+7885
+7886
+7887
+7888
+7889
+7890
+7891
+7892
+7893
+7894
+7895
+7896
+7897
+7898
+7899
+7900
+7901
+7902
+7903
+7904
+7905
+7906
+7907
+7908
+7909
+7910
+7911
+7912
+7913
+7914
+7915
+7916
+7917
+7918
+7919
+7920
+7921
+7922
+7923
+7924
+7925
+7926
+7927
+7928
+7929
+7930
+7931
+7932
+7933
+7934
+7935
+7936
+7937
+7938
+7939
+7940
+7941
+7942
+7943
+7944
+7945
+7946
+7947
+7948
+7949
+7950
+7951
+7952
+7953
+7954
+7955
+7956
+7957
+7958
+7959
+7960
+7961
+7962
+7963
+7964
+7965
+7966
+7967
+7968
+7969
+7970
+7971
+7972
+7973
+7974
+7975
+7976
+7977
+7978
+7979
+7980
+7981
+7982
+7983
+7984
+7985
+7986
+7987
+7988
+7989
+7990
+7991
+7992
+7993
+7994
+7995
+7996
+7997
+7998
+7999
+8000
+8001
+8002
+8003
+8004
+8005
+8006
+8007
+8008
+8009
+8010
+8011
+8012
+8013
+8014
+8015
+8016
+8017
+8018
+8019
+8020
+8021
+8022
+8023
+8024
+8025
+8026
+8027
+8028
+8029
+8030
+8031
+8032
+8033
+8034
+8035
+8036
+8037
+8038
+8039
+8040
+8041
+8042
+8043
+8044
+8045
+8046
+8047
+8048
+8049
+8050
+8051
+8052
+8053
+8054
+8055
+8056
+8057
+8058
+8059
+8060
+8061
+8062
+8063
+8064
+8065
+8066
+8067
+8068
+8069
+8070
+8071
+8072
+8073
+8074
+8075
+8076
+8077
+8078
+8079
+8080
+8081
+8082
+8083
+8084
+8085
+8086
+8087
+8088
+8089
+8090
+8091
+8092
+8093
+8094
+8095
+8096
+8097
+8098
+8099
+8100
+8101
+8102
+8103
+8104
+8105
+8106
+8107
+8108
+8109
+8110
+8111
+8112
+8113
+8114
+8115
+8116
+8117
+8118
+8119
+8120
+8121
+8122
+8123
+8124
+8125
+8126
+8127
+8128
+8129
+8130
+8131
+8132
+8133
+8134
+8135
+8136
+8137
+8138
+8139
+8140
+8141
+8142
+8143
+8144
+8145
+8146
+8147
+8148
+8149
+8150
+8151
+8152
+8153
+8154
+8155
+8156
+8157
+8158
+8159
+8160
+8161
+8162
+8163
+8164
+8165
+8166
+8167
+8168
+8169
+8170
+8171
+8172
+8173
+8174
+8175
+8176
+8177
+8178
+8179
+8180
+8181
+8182
+8183
+8184
+8185
+8186
+8187
+8188
+8189
+8190
+8191
+8192
+8193
+8194
+8195
+8196
+8197
+8198
+8199
+8200
+8201
+8202
+8203
+8204
+8205
+8206
+8207
+8208
+8209
+8210
+8211
+8212
+8213
+8214
+8215
+8216
+8217
+8218
+8219
+8220
+8221
+8222
+8223
+8224
+8225
+8226
+8227
+8228
+8229
+8230
+8231
+8232
+8233
+8234
+8235
+8236
+8237
+8238
+8239
+8240
+8241
+8242
+8243
+8244
+8245
+8246
+8247
+8248
+8249
+8250
+8251
+8252
+8253
+8254
+8255
+8256
+8257
+8258
+8259
+8260
+8261
+8262
+8263
+8264
+8265
+8266
+8267
+8268
+8269
+8270
+8271
+8272
+8273
+8274
+8275
+8276
+8277
+8278
+8279
+8280
+8281
+8282
+8283
+8284
+8285
+8286
+8287
+8288
+8289
+8290
+8291
+8292
+8293
+8294
+8295
+8296
+8297
+8298
+8299
+8300
+8301
+8302
+8303
+8304
+8305
+8306
+8307
+8308
+8309
+8310
+8311
+8312
+8313
+8314
+8315
+8316
+8317
+8318
+8319
+8320
+8321
+8322
+8323
+8324
+8325
+8326
+8327
+8328
+8329
+8330
+8331
+8332
+8333
+8334
+8335
+8336
+8337
+8338
+8339
+8340
+8341
+8342
+8343
+8344
+8345
+8346
+8347
+8348
+8349
+8350
+8351
+8352
+8353
+8354
+8355
+8356
+8357
+8358
+8359
+8360
+8361
+8362
+8363
+8364
+8365
+8366
+8367
+8368
+8369
+8370
+8371
+8372
+8373
+8374
+8375
+8376
+8377
+8378
+8379
+8380
+8381
+8382
+8383
+8384
+8385
+8386
+8387
+8388
+8389
+8390
+8391
+8392
+8393
+8394
+8395
+8396
+8397
+8398
+8399
+8400
+8401
+8402
+8403
+8404
+8405
+8406
+8407
+8408
+8409
+8410
+8411
+8412
+8413
+8414
+8415
+8416
+8417
+8418
+8419
+8420
+8421
+8422
+8423
+8424
+8425
+8426
+8427
+8428
+8429
+8430
+8431
+8432
+8433
+8434
+8435
+8436
+8437
+8438
+8439
+8440
+8441
+8442
+8443
+8444
+8445
+8446
+8447
+8448
+8449
+8450
+8451
+8452
+8453
+8454
+8455
+8456
+8457
+8458
+8459
+8460
+8461
+8462
+8463
+8464
+8465
+8466
+8467
+8468
+8469
+8470
+8471
+8472
+8473
+8474
+8475
+8476
+8477
+8478
+8479
+8480
+8481
+8482
+8483
+8484
+8485
+8486
+8487
+8488
+8489
+8490
+8491
+8492
+8493
+8494
+8495
+8496
+8497
+8498
+8499
+8500
+8501
+8502
+8503
+8504
+8505
+8506
+8507
+8508
+8509
+8510
+8511
+8512
+8513
+8514
+8515
+8516
+8517
+8518
+8519
+8520
+8521
+8522
+8523
+8524
+8525
+8526
+8527
+8528
+8529
+8530
+8531
+8532
+8533
+8534
+8535
+8536
+8537
+8538
+8539
+8540
+8541
+8542
+8543
+8544
+8545
+8546
+8547
+8548
+8549
+8550
+8551
+8552
+8553
+8554
+8555
+8556
+8557
+8558
+8559
+8560
+8561
+8562
+8563
+8564
+8565
+8566
+8567
+8568
+8569
+8570
+8571
+8572
+8573
+8574
+8575
+8576
+8577
+8578
+8579
+8580
+8581
+8582
+8583
+8584
+8585
+8586
+8587
+8588
+8589
+8590
+8591
+8592
+8593
+8594
+8595
+8596
+8597
+8598
+8599
+8600
+8601
+8602
+8603
+8604
+8605
+8606
+8607
+8608
+8609
+8610
+8611
+8612
+8613
+8614
+8615
+8616
+8617
+8618
+8619
+8620
+8621
+8622
+8623
+8624
+8625
+8626
+8627
+8628
+8629
+8630
+8631
+8632
+8633
+8634
+8635
+8636
+8637
+8638
+8639
+8640
+8641
+8642
+8643
+8644
+8645
+8646
+8647
+8648
+8649
+8650
+8651
+8652
+8653
+8654
+8655
+8656
+8657
+8658
+8659
+8660
+8661
+8662
+8663
+8664
+8665
+8666
+8667
+8668
+8669
+8670
+8671
+8672
+8673
+8674
+8675
+8676
+8677
+8678
+8679
+8680
+8681
+8682
+8683
+8684
+8685
+8686
+8687
+8688
+8689
+8690
+8691
+8692
+8693
+8694
+8695
+8696
+8697
+8698
+8699
+8700
+8701
+8702
+8703
+8704
+8705
+8706
+8707
+8708
+8709
+8710
+8711
+8712
+8713
+8714
+8715
+8716
+8717
+8718
+8719
+8720
+8721
+8722
+8723
+8724
+8725
+8726
+8727
+8728
+8729
+8730
+8731
+8732
+8733
+8734
+8735
+8736
+8737
+8738
+8739
+8740
+8741
+8742
+8743
+8744
+8745
+8746
+8747
+8748
+8749
+8750
+8751
+8752
+8753
+8754
+8755
+8756
+8757
+8758
+8759
+8760
+8761
+8762
+8763
+8764
+8765
+8766
+8767
+8768
+8769
+8770
+8771
+8772
+8773
+8774
+8775
+8776
+8777
+8778
+8779
+8780
+8781
+8782
+8783
+8784
+8785
+8786
+8787
+8788
+8789
+8790
+8791
+8792
+8793
+8794
+8795
+8796
+8797
+8798
+8799
+8800
+8801
+8802
+8803
+8804
+8805
+8806
+8807
+8808
+8809
+8810
+8811
+8812
+8813
+8814
+8815
+8816
+8817
+8818
+8819
+8820
+8821
+8822
+8823
+8824
+8825
+8826
+8827
+8828
+8829
+8830
+8831
+8832
+8833
+8834
+8835
+8836
+8837
+8838
+8839
+8840
+8841
+8842
+8843
+8844
+8845
+8846
+8847
+8848
+8849
+8850
+8851
+8852
+8853
+8854
+8855
+8856
+8857
+8858
+8859
+8860
+8861
+8862
+8863
+8864
+8865
+8866
+8867
+8868
+8869
+8870
+8871
+8872
+8873
+8874
+8875
+8876
+8877
+8878
+8879
+8880
+8881
+8882
+8883
+8884
+8885
+8886
+8887
+8888
+8889
+8890
+8891
+8892
+8893
+8894
+8895
+8896
+8897
+8898
+8899
+8900
+8901
+8902
+8903
+8904
+8905
+8906
+8907
+8908
+8909
+8910
+8911
+8912
+8913
+8914
+8915
+8916
+8917
+8918
+8919
+8920
+8921
+8922
+8923
+8924
+8925
+8926
+8927
+8928
+8929
+8930
+8931
+8932
+8933
+8934
+8935
+8936
+8937
+8938
+8939
+8940
+8941
+8942
+8943
+8944
+8945
+8946
+8947
+8948
+8949
+8950
+8951
+8952
+8953
+8954
+8955
+8956
+8957
+8958
+8959
+8960
+8961
+8962
+8963
+8964
+8965
+8966
+8967
+8968
+8969
+8970
+8971
+8972
+8973
+8974
+8975
+8976
+8977
+8978
+8979
+8980
+8981
+8982
+8983
+8984
+8985
+8986
+8987
+8988
+8989
+8990
+8991
+8992
+8993
+8994
+8995
+8996
+8997
+8998
+8999
+9000
+9001
+9002
+9003
+9004
+9005
+9006
+9007
+9008
+9009
+9010
+9011
+9012
+9013
+9014
+9015
+9016
+9017
+9018
+9019
+9020
+9021
+9022
+9023
+9024
+9025
+9026
+9027
+9028
+9029
+9030
+9031
+9032
+9033
+9034
+9035
+9036
+9037
+9038
+9039
+9040
+9041
+9042
+9043
+9044
+9045
+9046
+9047
+9048
+9049
+9050
+9051
+9052
+9053
+9054
+9055
+9056
+9057
+9058
+9059
+9060
+9061
+9062
+9063
+9064
+9065
+9066
+9067
+9068
+9069
+9070
+9071
+9072
+9073
+9074
+9075
+9076
+9077
+9078
+9079
+9080
+9081
+9082
+9083
+9084
+9085
+9086
+9087
+9088
+9089
+9090
+9091
+9092
+9093
+9094
+9095
+9096
+9097
+9098
+9099
+9100
+9101
+9102
+9103
+9104
+9105
+9106
+9107
+9108
+9109
+9110
+9111
+9112
+9113
+9114
+9115
+9116
+9117
+9118
+9119
+9120
+9121
+9122
+9123
+9124
+9125
+9126
+9127
+9128
+9129
+9130
+9131
+9132
+9133
+9134
+9135
+9136
+9137
+9138
+9139
+9140
+9141
+9142
+9143
+9144
+9145
+9146
+9147
+9148
+9149
+9150
+9151
+9152
+9153
+9154
+9155
+9156
+9157
+9158
+9159
+9160
+9161
+9162
+9163
+9164
+9165
+9166
+9167
+9168
+9169
+9170
+9171
+9172
+9173
+9174
+9175
+9176
+9177
+9178
+9179
+9180
+9181
+9182
+9183
+9184
+9185
+9186
+9187
+9188
+9189
+9190
+9191
+9192
+9193
+9194
+9195
+9196
+9197
+9198
+9199
+9200
+9201
+9202
+9203
+9204
+9205
+9206
+9207
+9208
+9209
+9210
+9211
+9212
+9213
+9214
+9215
+9216
+9217
+9218
+9219
+9220
+9221
+9222
+9223
+9224
+9225
+9226
+9227
+9228
+9229
+9230
+9231
+9232
+9233
+9234
+9235
+9236
+9237
+9238
+9239
+9240
+9241
+9242
+9243
+9244
+9245
+9246
+9247
+9248
+9249
+9250
+9251
+9252
+9253
+9254
+9255
+9256
+9257
+9258
+9259
+9260
+9261
+9262
+9263
+9264
+9265
+9266
+9267
+9268
+9269
+9270
+9271
+9272
+9273
+9274
+9275
+9276
+9277
+9278
+9279
+9280
+9281
+9282
+9283
+9284
+9285
+9286
+9287
+9288
+9289
+9290
+9291
+9292
+9293
+9294
+9295
+9296
+9297
+9298
+9299
+9300
+9301
+9302
+9303
+9304
+9305
+9306
+9307
+9308
+9309
+9310
+9311
+9312
+9313
+9314
+9315
+9316
+9317
+9318
+9319
+9320
+9321
+9322
+9323
+9324
+9325
+9326
+9327
+9328
+9329
+9330
+9331
+9332
+9333
+9334
+9335
+9336
+9337
+9338
+9339
+9340
+9341
+9342
+9343
+9344
+9345
+9346
+9347
+9348
+9349
+9350
+9351
+9352
+9353
+9354
+9355
+9356
+9357
+9358
+9359
+9360
+9361
+9362
+9363
+9364
+9365
+9366
+9367
+9368
+9369
+9370
+9371
+9372
+9373
+9374
+9375
+9376
+9377
+9378
+9379
+9380
+9381
+9382
+9383
+9384
+9385
+9386
+9387
+9388
+9389
+9390
+9391
+9392
+9393
+9394
+9395
+9396
+9397
+9398
+9399
+9400
+9401
+9402
+9403
+9404
+9405
+9406
+9407
+9408
+9409
+9410
+9411
+9412
+9413
+9414
+9415
+9416
+9417
+9418
+9419
+9420
+9421
+9422
+9423
+9424
+9425
+9426
+9427
+9428
+9429
+9430
+9431
+9432
+9433
+9434
+9435
+9436
+9437
+9438
+9439
+9440
+9441
+9442
+9443
+9444
+9445
+9446
+9447
+9448
+9449
+9450
+9451
+9452
+9453
+9454
+9455
+9456
+9457
+9458
+9459
+9460
+9461
+9462
+9463
+9464
+9465
+9466
+9467
+9468
+9469
+9470
+9471
+9472
+9473
+9474
+9475
+9476
+9477
+9478
+9479
+9480
+9481
+9482
+9483
+9484
+9485
+9486
+9487
+9488
+9489
+9490
+9491
+9492
+9493
+9494
+9495
+9496
+9497
+9498
+9499
+9500
+9501
+9502
+9503
+9504
+9505
+9506
+9507
+9508
+9509
+9510
+9511
+9512
+9513
+9514
+9515
+9516
+9517
+9518
+9519
+9520
+9521
+9522
+9523
+9524
+9525
+9526
+9527
+9528
+9529
+9530
+9531
+9532
+9533
+9534
+9535
+9536
+9537
+9538
+9539
+9540
+9541
+9542
+9543
+9544
+9545
+9546
+9547
+9548
+9549
+9550
+9551
+9552
+9553
+9554
+9555
+9556
+9557
+9558
+9559
+9560
+9561
+9562
+9563
+9564
+9565
+9566
+9567
+9568
+9569
+9570
+9571
+9572
+9573
+9574
+9575
+9576
+9577
+9578
+9579
+9580
+9581
+9582
+9583
+9584
+9585
+9586
+9587
+9588
+9589
+9590
+9591
+9592
+9593
+9594
+9595
+9596
+9597
+9598
+9599
+9600
+9601
+9602
+9603
+9604
+9605
+9606
+9607
+9608
+9609
+9610
+9611
+9612
+9613
+9614
+9615
+9616
+9617
+9618
+9619
+9620
+9621
+9622
+9623
+9624
+9625
+9626
+9627
+9628
+9629
+9630
+9631
+9632
+9633
+9634
+9635
+9636
+9637
+9638
+9639
+9640
+9641
+9642
+9643
+9644
+9645
+9646
+9647
+9648
+9649
+9650
+9651
+9652
+9653
+9654
+9655
+9656
+9657
+9658
+9659
+9660
+9661
+9662
+9663
+9664
+9665
+9666
+9667
+9668
+9669
+9670
+9671
+9672
+9673
+9674
+9675
+9676
+9677
+9678
+9679
+9680
+9681
+9682
+9683
+9684
+9685
+9686
+9687
+9688
+9689
+9690
+9691
+9692
+9693
+9694
+9695
+9696
+9697
+9698
+9699
+9700
+9701
+9702
+9703
+9704
+9705
+9706
+9707
+9708
+9709
+9710
+9711
+9712
+9713
+9714
+9715
+9716
+9717
+9718
+9719
+9720
+9721
+9722
+9723
+9724
+9725
+9726
+9727
+9728
+9729
+9730
+9731
+9732
+9733
+9734
+9735
+9736
+9737
+9738
+9739
+9740
+9741
+9742
+9743
+9744
+9745
+9746
+9747
+9748
+9749
+9750
+9751
+9752
+9753
+9754
+9755
+9756
+9757
+9758
+9759
+9760
+9761
+9762
+9763
+9764
+9765
+9766
+9767
+9768
+9769
+9770
+9771
+9772
+9773
+9774
+9775
+9776
+9777
+9778
+9779
+9780
+9781
+9782
+9783
+9784
+9785
+9786
+9787
+9788
+9789
+9790
+9791
+9792
+9793
+9794
+9795
+9796
+9797
+9798
+9799
+9800
+9801
+9802
+9803
+9804
+9805
+9806
+9807
+9808
+9809
+9810
+9811
+9812
+9813
+9814
+9815
+9816
+9817
+9818
+9819
+9820
+9821
+9822
+9823
+9824
+9825
+9826
+9827
+9828
+9829
+9830
+9831
+9832
+9833
+9834
+9835
+9836
+9837
+9838
+9839
+9840
+9841
+9842
+9843
+9844
+9845
+9846
+9847
+9848
+9849
+9850
+9851
+9852
+9853
+9854
+9855
+9856
+9857
+9858
+9859
+9860
+9861
+9862
+9863
+9864
+9865
+9866
+9867
+9868
+9869
+9870
+9871
+9872
+9873
+9874
+9875
+9876
+9877
+9878
+9879
+9880
+9881
+9882
+9883
+9884
+9885
+9886
+9887
+9888
+9889
+9890
+9891
+9892
+9893
+9894
+9895
+9896
+9897
+9898
+9899
+9900
+9901
+9902
+9903
+9904
+9905
+9906
+9907
+9908
+9909
+9910
+9911
+9912
+9913
+9914
+9915
+9916
+9917
+9918
+9919
+9920
+9921
+9922
+9923
+9924
+9925
+9926
+9927
+9928
+9929
+9930
+9931
+9932
+9933
+9934
+9935
+9936
+9937
+9938
+9939
+9940
+9941
+9942
+9943
+9944
+9945
+9946
+9947
+9948
+9949
+9950
+9951
+9952
+9953
+9954
+9955
+9956
+9957
+9958
+9959
+9960
+9961
+9962
+9963
+9964
+9965
+9966
+9967
+9968
+9969
+9970
+9971
+9972
+9973
+9974
+9975
+9976
+9977
+9978
+9979
+9980
+9981
+9982
+9983
+9984
+9985
+9986
+9987
+9988
+9989
+9990
+9991
+9992
+9993
+9994
+9995
+9996
+9997
+9998
+9999
+10000
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 a866c1c..2508a1e 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
@@ -218,6 +218,7 @@
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setFixedVertexValueSize(true);
         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 1bb33b8..fa16ce5 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
@@ -87,6 +87,7 @@
 
     private ByteWritable tmpVertexValue = new ByteWritable();
     private long sourceId = -1;
+    private long destId = -1;
 
     /** The source vertex id */
     public static final String SOURCE_ID = "ReachibilityVertex.sourceId";
@@ -112,7 +113,7 @@
      * @return True if the source id
      */
     private boolean isDest(VLongWritable v) {
-        return (v.get() == getContext().getConfiguration().getLong(DEST_ID, DEST_ID_DEFAULT));
+        return (v.get() == destId);
     }
 
     @Override
@@ -120,6 +121,9 @@
         if (sourceId < 0) {
             sourceId = getContext().getConfiguration().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
         }
+        if (destId < 0) {
+            destId = getContext().getConfiguration().getLong(DEST_ID, DEST_ID_DEFAULT);
+        }
         if (getSuperstep() == 1) {
             boolean isSource = isSource(getVertexId());
             if (isSource) {
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 648f168..2fea813 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
@@ -16,8 +16,6 @@
 package edu.uci.ics.pregelix.example;
 
 import java.util.Iterator;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 import org.apache.hadoop.io.FloatWritable;
 
@@ -84,12 +82,12 @@
 
     private DoubleWritable outputValue = new DoubleWritable();
     private DoubleWritable tmpVertexValue = new DoubleWritable();
-    /** Class logger */
-    private static final Logger LOG = Logger.getLogger(ShortestPathsVertex.class.getName());
     /** The shortest paths id */
     public static final String SOURCE_ID = "SimpleShortestPathsVertex.sourceId";
     /** Default shortest paths id */
     public static final long SOURCE_ID_DEFAULT = 1;
+    /** the source vertex id */
+    private long sourceId = -1;
 
     /**
      * Is this vertex the source id?
@@ -97,11 +95,14 @@
      * @return True if the source id
      */
     private boolean isSource() {
-        return (getVertexId().get() == getContext().getConfiguration().getLong(SOURCE_ID, SOURCE_ID_DEFAULT));
+        return (getVertexId().get() == sourceId);
     }
 
     @Override
     public void compute(Iterator<DoubleWritable> msgIterator) {
+        if (sourceId < 0) {
+            sourceId = getContext().getConfiguration().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+        }
         if (getSuperstep() == 1) {
             tmpVertexValue.set(Double.MAX_VALUE);
             setVertexValue(tmpVertexValue);
@@ -110,17 +111,10 @@
         while (msgIterator.hasNext()) {
             minDist = Math.min(minDist, msgIterator.next().get());
         }
-        if (LOG.getLevel() == Level.FINE) {
-            LOG.fine("Vertex " + getVertexId() + " got minDist = " + minDist + " vertex value = " + getVertexValue());
-        }
         if (minDist < getVertexValue().get()) {
             tmpVertexValue.set(minDist);
             setVertexValue(tmpVertexValue);
             for (Edge<VLongWritable, FloatWritable> edge : getEdges()) {
-                if (LOG.getLevel() == Level.FINE) {
-                    LOG.fine("Vertex " + getVertexId() + " sent to " + edge.getDestVertexId() + " = "
-                            + (minDist + edge.getEdgeValue().get()));
-                }
                 outputValue.set(minDist + edge.getEdgeValue().get());
                 sendMsg(edge.getDestVertexId(), outputValue);
             }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextShortestPathsInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextShortestPathsInputFormat.java
index 3ea4a9f..8987393 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextShortestPathsInputFormat.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextShortestPathsInputFormat.java
@@ -73,6 +73,7 @@
 
         vertex.getMsgList().clear();
         vertex.getEdges().clear();
+        vertex.reset();
         Text line = getRecordReader().getCurrentValue();
         String[] fields = line.toString().split(separator);
 
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java
index ff1e29f..421f2f5 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java
@@ -56,6 +56,7 @@
             FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
             job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
             job.setCheckpointHook(ConservativeCheckpointHook.class);
+            job.setFixedVertexValueSize(true);
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
index 3fdaf15..b3ad112 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
@@ -55,6 +55,7 @@
             FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
             job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
             job.setCheckpointHook(ConservativeCheckpointHook.class);
+            job.setFixedVertexValueSize(true);
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java
index e006ccd..9a2ef2c 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java
@@ -53,6 +53,7 @@
             FileInputFormat.setInputPaths(job, INPUTPATH);
             FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
             job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+            job.setFixedVertexValueSize(true);
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobConnectedComponentsTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobConnectedComponentsTest.java
new file mode 100644
index 0000000..65b9845
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobConnectedComponentsTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
+import edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * This test case tests multi-user workload, using ConnectedComponents.
+ * 
+ * @author yingyib
+ */
+public class MultiJobConnectedComponentsTest {
+    private static String INPUTPATH = "data/webmapcomplex";
+    private static String OUTPUTPAH = "actual/result";
+    private static String OUTPUTPAH2 = "actual/result2";
+    private static String EXPECTEDPATH = "src/test/resources/expected/ConnectedComponentsRealComplex";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+        try {
+            PregelixJob job = new PregelixJob(ConnectedComponentsVertex.class.getName());
+            job.setVertexClass(ConnectedComponentsVertex.class);
+            job.setVertexClass(ConnectedComponentsVertex.class);
+            job.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class);
+            job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
+            job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+            job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+            job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+            job.setDynamicVertexValueSize(true);
+            FileInputFormat.setInputPaths(job, INPUTPATH);
+            FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+            job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
+            job.setCheckpointHook(ConservativeCheckpointHook.class);
+
+            testCluster.setUp();
+            Thread thread = new Thread(new Runnable() {
+
+                @Override
+                public void run() {
+                    try {
+                        Driver driver = new Driver(PageRankVertex.class);
+                        PregelixJob job2 = new PregelixJob(ConnectedComponentsVertex.class.getName());
+                        job2.setVertexClass(ConnectedComponentsVertex.class);
+                        job2.setVertexClass(ConnectedComponentsVertex.class);
+                        job2.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class);
+                        job2.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
+                        job2.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+                        job2.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+                        job2.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+                        job2.setDynamicVertexValueSize(true);
+                        FileInputFormat.setInputPaths(job2, INPUTPATH);
+                        FileOutputFormat.setOutputPath(job2, new Path(OUTPUTPAH2));
+                        job2.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
+                        job2.setCheckpointHook(ConservativeCheckpointHook.class);
+                        driver.runJob(job2, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+                        TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH2));
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            });
+            thread.start();
+            Driver driver = new Driver(PageRankVertex.class);
+            driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+            TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+            thread.join();
+        } catch (Exception e) {
+            PregelixHyracksIntegrationUtil.deinit();
+            testCluster.cleanupHDFS();
+            throw e;
+        }
+    }
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobPageRankTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobPageRankTest.java
new file mode 100644
index 0000000..cfd1b27
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobPageRankTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * This test case tests multi-user workload, using PageRank.
+ * 
+ * @author yingyib
+ */
+public class MultiJobPageRankTest {
+    private static String INPUTPATH = "data/webmap";
+    private static String OUTPUTPAH = "actual/result";
+    private static String OUTPUTPAH2 = "actual/result2";
+    private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+        try {
+            PregelixJob job = new PregelixJob(PageRankVertex.class.getName());
+            job.setVertexClass(PageRankVertex.class);
+            job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+            job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+            job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+            job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+            FileInputFormat.setInputPaths(job, INPUTPATH);
+            FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+            job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+            job.setCheckpointHook(ConservativeCheckpointHook.class);
+            job.setFixedVertexValueSize(true);
+
+            testCluster.setUp();
+            Thread thread = new Thread(new Runnable() {
+
+                @Override
+                public void run() {
+                    try {
+                        Driver driver = new Driver(PageRankVertex.class);
+                        PregelixJob job2 = new PregelixJob(PageRankVertex.class.getName());
+                        job2.setVertexClass(PageRankVertex.class);
+                        job2.setVertexInputFormatClass(TextPageRankInputFormat.class);
+                        job2.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+                        job2.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+                        job2.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+                        FileInputFormat.setInputPaths(job2, INPUTPATH);
+                        FileOutputFormat.setOutputPath(job2, new Path(OUTPUTPAH2));
+                        job2.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+                        job2.setCheckpointHook(ConservativeCheckpointHook.class);
+                        job2.setFixedVertexValueSize(true);
+                        driver.runJob(job2, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+                        TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH2));
+                    } catch (Exception e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            });
+            thread.start();
+            Driver driver = new Driver(PageRankVertex.class);
+            driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+            TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+            thread.join();
+        } catch (Exception e) {
+            PregelixHyracksIntegrationUtil.deinit();
+            testCluster.cleanupHDFS();
+            throw e;
+        }
+    }
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertex.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertex.java
new file mode 100644
index 0000000..1e6359d
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertex.java
@@ -0,0 +1,106 @@
+/*
+ * 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;
+
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * @author yingyib
+ */
+public class UpdateVertex extends Vertex<VLongWritable, Text, FloatWritable, VLongWritable> {
+    private final int MAX_VALUE_SIZE = 32768 / 2;
+    private VLongWritable msg = new VLongWritable();
+    private Text tempValue = new Text();
+    private Random rand = new Random();
+
+    @Override
+    public void compute(Iterator<VLongWritable> msgIterator) throws Exception {
+        if (getSuperstep() == 1) {
+            updateAndSendMsg();
+        } else if (getSuperstep() > 1 && getSuperstep() < 2) {
+            verifyVertexSize(msgIterator);
+            updateAndSendMsg();
+        } else {
+            voteToHalt();
+        }
+    }
+
+    private void verifyVertexSize(Iterator<VLongWritable> msgIterator) {
+        if (!msgIterator.hasNext()) {
+            throw new IllegalStateException("no message for vertex " + " " + getVertexId() + " " + getVertexValue());
+        }
+        /**
+         * verify the size
+         */
+        int valueSize = getVertexValue().toString().toCharArray().length;
+        long expectedValueSize = msgIterator.next().get();
+        if (valueSize != expectedValueSize) {
+            if (valueSize == -expectedValueSize) {
+                //verify fixed size update
+                char[] valueCharArray = getVertexValue().toString().toCharArray();
+                for (int i = 0; i < valueCharArray.length; i++) {
+                    if (valueCharArray[i] != 'b') {
+                        throw new IllegalStateException("vertex id: " + getVertexId()
+                                + " has a un-propagated update in the last iteration");
+                    }
+                }
+            } else {
+                throw new IllegalStateException("vertex id: " + getVertexId() + " vertex value size:" + valueSize
+                        + ", expected value size:" + expectedValueSize);
+            }
+        }
+        if (msgIterator.hasNext()) {
+            throw new IllegalStateException("more than one message for vertex " + " " + getVertexId() + " "
+                    + getVertexValue());
+        }
+    }
+
+    private void updateAndSendMsg() {
+        int newValueSize = rand.nextInt(MAX_VALUE_SIZE);
+        char[] charArray = new char[newValueSize];
+        for (int i = 0; i < charArray.length; i++) {
+            charArray[i] = 'a';
+        }
+        /**
+         * set a self-message
+         */
+        msg.set(newValueSize);
+        boolean fixedSize = getVertexId().get() < 2000;
+        if (fixedSize) {
+            int oldSize = getVertexValue().toString().toCharArray().length;
+            charArray = new char[oldSize];
+            for (int i = 0; i < oldSize; i++) {
+                charArray[i] = 'b';
+            }
+            msg.set(-oldSize);
+        }
+
+        /**
+         * set the vertex value
+         */
+        tempValue.set(new String(charArray));
+        setVertexValue(tempValue);
+
+        sendMsg(getVertexId(), msg);
+        activate();
+    }
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexInputFormat.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexInputFormat.java
new file mode 100644
index 0000000..076337a
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexInputFormat.java
@@ -0,0 +1,112 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexInputFormat;
+import edu.uci.ics.pregelix.api.io.VertexReader;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.example.io.DoubleWritable;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class UpdateVertexInputFormat extends VertexInputFormat<VLongWritable, Text, FloatWritable, DoubleWritable> {
+
+    @Override
+    public VertexReader<VLongWritable, Text, FloatWritable, DoubleWritable> createVertexReader(InputSplit split,
+            TaskAttemptContext context) throws IOException {
+        return new UpdateVertexReader(context);
+    }
+
+    @Override
+    public List<InputSplit> getSplits(JobContext context, int numWorkers) throws IOException, InterruptedException {
+        InputSplit split = new FileSplit(new Path("testdata"), 0, 0, new String[0]);
+        return Collections.singletonList(split);
+    }
+}
+
+@SuppressWarnings("rawtypes")
+class UpdateVertexReader implements VertexReader<VLongWritable, Text, FloatWritable, DoubleWritable> {
+
+    private final static int MAX_ID = 65536;
+    private final static int MIN_ID = -65536;
+    private Vertex vertex;
+    private VLongWritable vertexId = new VLongWritable();
+    private int currentId = MIN_ID;
+    private TaskAttemptContext context;
+
+    public UpdateVertexReader(TaskAttemptContext context) {
+        this.context = context;
+    }
+
+    private TaskAttemptContext getContext() {
+        return context;
+    }
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+        return currentId < MAX_ID;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Vertex<VLongWritable, Text, FloatWritable, DoubleWritable> getCurrentVertex() throws IOException,
+            InterruptedException {
+        if (vertex == null)
+            vertex = (Vertex) BspUtils.createVertex(getContext().getConfiguration());
+        vertex.getMsgList().clear();
+        vertex.getEdges().clear();
+
+        vertex.reset();
+
+        /**
+         * set the src vertex id
+         */
+        vertexId.set(currentId++);
+        vertex.setVertexId(vertexId);
+
+        /**
+         * set the vertex value
+         */
+        vertex.setVertexValue(new Text("aaa"));
+        return vertex;
+    }
+
+    @Override
+    public void initialize(InputSplit inputSplit, TaskAttemptContext context) throws IOException, InterruptedException {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        return 0;
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexOutputFormat.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexOutputFormat.java
new file mode 100644
index 0000000..493a33e
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexOutputFormat.java
@@ -0,0 +1,54 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class UpdateVertexOutputFormat extends TextVertexOutputFormat<VLongWritable, Text, FloatWritable> {
+
+    @Override
+    public VertexWriter<VLongWritable, Text, FloatWritable> createVertexWriter(TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        final RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+        return new UpdateVertexWriter(recordWriter);
+    }
+
+    /**
+     * Simple VertexWriter that support
+     */
+    public static class UpdateVertexWriter extends TextVertexWriter<VLongWritable, Text, FloatWritable> {
+        public UpdateVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+            super(lineRecordWriter);
+        }
+
+        @Override
+        public void writeVertex(Vertex<VLongWritable, Text, FloatWritable, ?> vertex) throws IOException,
+                InterruptedException {
+            int len = vertex.getVertexValue().toString().toCharArray().length;
+            getRecordWriter().write(new Text(vertex.getVertexId().toString()), new Text(Integer.toString(len)));
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexTest.java
new file mode 100644
index 0000000..6adc5bb
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertexTest.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;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.core.base.IDriver.Plan;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+
+/**
+ * This test case tests the error message propagation.
+ * 
+ * @author yingyib
+ */
+public class UpdateVertexTest {
+
+    private static String INPUT_PATH = "/data/webmap/";
+    private static String OUTPUT_PATH = "actual/resultcomplex";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+        try {
+            PregelixJob job = new PregelixJob(UpdateVertex.class.getSimpleName());
+            job.setVertexClass(UpdateVertex.class);
+            job.setVertexInputFormatClass(UpdateVertexInputFormat.class);
+            job.setVertexOutputFormatClass(UpdateVertexOutputFormat.class);
+            job.setDynamicVertexValueSize(true);
+
+            FileInputFormat.setInputPaths(job, INPUT_PATH);
+            FileOutputFormat.setOutputPath(job, new Path(OUTPUT_PATH));
+            job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
+
+            Driver driver = new Driver(UpdateVertex.class);
+            testCluster.setUp();
+
+            Plan[] plans = new Plan[] { Plan.INNER_JOIN, Plan.OUTER_JOIN };
+            for (Plan plan : plans) {
+                driver.runJob(job, plan, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        } finally {
+            testCluster.tearDown();
+        }
+    }
+
+}
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 f6857fe..2dfd071 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
@@ -74,6 +74,7 @@
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setFixedVertexValueSize(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -89,6 +90,7 @@
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+        job.setFixedVertexValueSize(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -116,6 +118,7 @@
         job.setVertexInputFormatClass(TextPageRankInputFormat.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setFixedVertexValueSize(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -190,6 +193,7 @@
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setFixedVertexValueSize(true);
         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/resources/jobs/PageRank.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
index 65e0b30..d29b2da 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml
@@ -127,6 +127,7 @@
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimulatedPageRankVertexInputFormat</value></property>
 <property><name>mapred.job.queue.name</name><value>default</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>false</value></property>
 <property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
 <property><name>topology.script.number.args</name><value>100</value></property>
 <property><name>mapred.skip.map.max.skip.records</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 b50b02a..69f8154 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
@@ -128,6 +128,7 @@
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
 <property><name>mapred.job.queue.name</name><value>default</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>false</value></property>
 <property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
 <property><name>topology.script.number.args</name><value>100</value></property>
 <property><name>mapred.skip.map.max.skip.records</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 217fbba..04fc686 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
@@ -1,145 +1,146 @@
 <?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
-<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
-<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
-<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
-<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
-<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
-<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
-<property><name>mapred.input.dir</name><value>file:/webmapcomplex</value></property>
-<property><name>mapred.submit.replication</name><value>10</value></property>
-<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
-<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
-<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
-<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
-<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
-<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
-<property><name>keep.failed.task.files</name><value>false</value></property>
-<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
-<property><name>io.bytes.per.checksum</name><value>512</value></property>
-<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
-<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
-<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
-<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
-<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
-<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
-<property><name>fs.checkpoint.period</name><value>3600</value></property>
-<property><name>mapred.child.tmp</name><value>./tmp</value></property>
-<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
-<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
-<property><name>hadoop.logfile.count</name><value>10</value></property>
-<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
-<property><name>mapred.output.dir</name><value>/resultcomplex</value></property>
-<property><name>io.map.index.skip</name><value>0</value></property>
-<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
-<property><name>mapred.output.compress</name><value>false</value></property>
-<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
-<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
-<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>
-<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
-<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
-<property><name>mapred.task.timeout</name><value>600000</value></property>
-<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
-<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
-<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
-<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
-<property><name>ipc.client.kill.max</name><value>10</value></property>
-<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
-<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
-<property><name>io.sort.record.percent</name><value>0.05</value></property>
-<property><name>hadoop.security.authorization</name><value>false</value></property>
-<property><name>mapred.max.tracker.failures</name><value>4</value></property>
-<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
-<property><name>pregelix.numVertices</name><value>23</value></property>
-<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
-<property><name>mapred.map.tasks</name><value>2</value></property>
-<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
-<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
-<property><name>fs.default.name</name><value>file:///</value></property>
-<property><name>tasktracker.http.threads</name><value>40</value></property>
-<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
-<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
-<property><name>mapred.reduce.tasks</name><value>1</value></property>
-<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
-<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex</value></property>
-<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
-<property><name>io.file.buffer.size</name><value>4096</value></property>
-<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
-<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
-<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
-<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
-<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
-<property><name>hadoop.native.lib</name><value>true</value></property>
-<property><name>fs.s3.block.size</name><value>67108864</value></property>
-<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
-<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
-<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
-<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
-<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
-<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
-<property><name>mapred.queue.names</name><value>default</value></property>
-<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
-<property><name>pregelix.checkpointHook</name><value>edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook</value></property>
-<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
-<property><name>mapred.job.tracker</name><value>local</value></property>
-<property><name>io.skip.checksum.errors</name><value>false</value></property>
-<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
-<property><name>fs.s3.maxRetries</name><value>4</value></property>
-<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
-<property><name>fs.trash.interval</name><value>0</value></property>
-<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
-<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
-<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
-<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
-<property><name>io.sort.mb</name><value>100</value></property>
-<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
-<property><name>io.sort.factor</name><value>10</value></property>
-<property><name>mapred.task.profile</name><value>false</value></property>
-<property><name>job.end.retry.interval</name><value>30000</value></property>
-<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
-<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
-<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
-<property><name>webinterface.private.actions</name><value>false</value></property>
-<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
-<property><name>pregelix.combinerClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner</value></property>
-<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
-<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
-<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
-<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
-<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
-<property><name>mapred.compress.map.output</name><value>false</value></property>
-<property><name>io.sort.spill.percent</name><value>0.80</value></property>
-<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
-<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
-<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
-<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
-<property><name>job.end.retry.attempts</name><value>0</value></property>
-<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
-<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat</value></property>
-<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
-<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
-<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
-<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
-<property><name>hadoop.logfile.size</name><value>10000000</value></property>
-<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
-<property><name>mapred.job.queue.name</name><value>default</value></property>
-<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
-<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
-<property><name>topology.script.number.args</name><value>100</value></property>
-<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
-<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
 <property><name>mapred.task.cache.levels</name><value>2</value></property>
-<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
-<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
-<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>hadoop.tmp.dir</name><value>/tmp/hadoop-${user.name}</value></property>
+<property><name>hadoop.native.lib</name><value>true</value></property>
+<property><name>map.sort.class</name><value>org.apache.hadoop.util.QuickSort</value></property>
+<property><name>ipc.client.idlethreshold</name><value>4000</value></property>
+<property><name>mapred.system.dir</name><value>${hadoop.tmp.dir}/mapred/system</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.hours</name><value>0</value></property>
+<property><name>io.skip.checksum.errors</name><value>false</value></property>
+<property><name>fs.default.name</name><value>file:///</value></property>
+<property><name>mapred.child.tmp</name><value>./tmp</value></property>
+<property><name>fs.har.impl.disable.cache</name><value>true</value></property>
+<property><name>mapred.skip.reduce.max.skip.groups</name><value>0</value></property>
+<property><name>mapred.jobtracker.instrumentation</name><value>org.apache.hadoop.mapred.JobTrackerMetricsInst</value></property>
+<property><name>mapred.tasktracker.dns.nameserver</name><value>default</value></property>
+<property><name>io.sort.factor</name><value>10</value></property>
+<property><name>pregelix.checkpointHook</name><value>edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook</value></property>
+<property><name>mapred.task.timeout</name><value>600000</value></property>
+<property><name>mapred.max.tracker.failures</name><value>4</value></property>
+<property><name>hadoop.rpc.socket.factory.class.default</name><value>org.apache.hadoop.net.StandardSocketFactory</value></property>
+<property><name>fs.hdfs.impl</name><value>org.apache.hadoop.hdfs.DistributedFileSystem</value></property>
+<property><name>mapred.queue.default.acl-administer-jobs</name><value>*</value></property>
+<property><name>mapred.queue.default.acl-submit-job</name><value>*</value></property>
+<property><name>mapred.skip.map.auto.incr.proc.count</name><value>true</value></property>
+<property><name>io.mapfile.bloom.size</name><value>1048576</value></property>
+<property><name>tasktracker.http.threads</name><value>40</value></property>
+<property><name>mapred.job.shuffle.merge.percent</name><value>0.66</value></property>
+<property><name>fs.ftp.impl</name><value>org.apache.hadoop.fs.ftp.FTPFileSystem</value></property>
+<property><name>pregelix.combinerClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner</value></property>
+<property><name>mapred.output.compress</name><value>false</value></property>
+<property><name>io.bytes.per.checksum</name><value>512</value></property>
+<property><name>topology.node.switch.mapping.impl</name><value>org.apache.hadoop.net.ScriptBasedMapping</value></property>
+<property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
+<property><name>mapred.reduce.max.attempts</name><value>4</value></property>
+<property><name>fs.ramfs.impl</name><value>org.apache.hadoop.fs.InMemoryFileSystem</value></property>
+<property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.dir</name><value>/jobtracker/jobsInfo</value></property>
+<property><name>fs.s3.buffer.dir</name><value>${hadoop.tmp.dir}/s3</value></property>
+<property><name>job.end.retry.attempts</name><value>0</value></property>
+<property><name>fs.file.impl</name><value>org.apache.hadoop.fs.LocalFileSystem</value></property>
+<property><name>mapred.local.dir.minspacestart</name><value>0</value></property>
+<property><name>mapred.output.compression.type</name><value>RECORD</value></property>
+<property><name>topology.script.number.args</name><value>100</value></property>
+<property><name>io.mapfile.bloom.error.rate</name><value>0.005</value></property>
+<property><name>mapred.max.tracker.blacklists</name><value>4</value></property>
+<property><name>pregelix.partitionerClass</name><value>edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner</value></property>
+<property><name>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>mapred.userlog.retain.hours</name><value>24</value></property>
+<property><name>pregelix.numVertices</name><value>23</value></property>
+<property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>hadoop.security.authorization</name><value>false</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>mapred.min.split.size</name><value>0</value></property>
+<property><name>mapred.map.tasks</name><value>2</value></property>
+<property><name>mapred.child.java.opts</name><value>-Xmx200m</value></property>
+<property><name>mapred.job.queue.name</name><value>default</value></property>
+<property><name>ipc.server.listen.queue.size</name><value>128</value></property>
+<property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
+<property><name>job.end.retry.interval</name><value>30000</value></property>
+<property><name>mapred.skip.attempts.to.start.skipping</name><value>2</value></property>
+<property><name>fs.checkpoint.dir</name><value>${hadoop.tmp.dir}/dfs/namesecondary</value></property>
+<property><name>mapred.reduce.tasks</name><value>1</value></property>
+<property><name>mapred.merge.recordsBeforeProgress</name><value>10000</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<property><name>webinterface.private.actions</name><value>false</value></property>
+<property><name>io.sort.spill.percent</name><value>0.80</value></property>
+<property><name>mapred.job.shuffle.input.buffer.percent</name><value>0.70</value></property>
+<property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
+<property><name>mapred.job.name</name><value>PageRank</value></property>
+<property><name>hadoop.util.hash.type</name><value>murmur</value></property>
 <property><name>mapred.map.max.attempts</name><value>4</value></property>
-<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>pregelix.incStateLength</name><value>false</value></property>
+<property><name>mapred.job.tracker.handler.count</name><value>10</value></property>
+<property><name>mapred.tasktracker.expiry.interval</name><value>600000</value></property>
+<property><name>mapred.jobtracker.maxtasks.per.job</name><value>-1</value></property>
+<property><name>mapred.jobtracker.job.history.block.size</name><value>3145728</value></property>
+<property><name>keep.failed.task.files</name><value>false</value></property>
 <property><name>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.task.profile.reduces</name><value>0-2</value></property>
+<property><name>mapred.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>io.map.index.skip</name><value>0</value></property>
+<property><name>ipc.server.tcpnodelay</name><value>false</value></property>
+<property><name>hadoop.logfile.size</name><value>10000000</value></property>
+<property><name>mapred.reduce.tasks.speculative.execution</name><value>true</value></property>
+<property><name>fs.checkpoint.period</name><value>3600</value></property>
+<property><name>mapred.job.reuse.jvm.num.tasks</name><value>1</value></property>
+<property><name>mapred.jobtracker.completeuserjobs.maximum</name><value>100</value></property>
+<property><name>fs.s3.maxRetries</name><value>4</value></property>
+<property><name>mapred.local.dir</name><value>${hadoop.tmp.dir}/mapred/local</value></property>
+<property><name>fs.hftp.impl</name><value>org.apache.hadoop.hdfs.HftpFileSystem</value></property>
+<property><name>fs.trash.interval</name><value>0</value></property>
+<property><name>fs.s3.sleepTimeSeconds</name><value>10</value></property>
+<property><name>mapred.submit.replication</name><value>10</value></property>
+<property><name>fs.har.impl</name><value>org.apache.hadoop.fs.HarFileSystem</value></property>
+<property><name>mapred.map.output.compression.codec</name><value>org.apache.hadoop.io.compress.DefaultCodec</value></property>
+<property><name>mapred.tasktracker.dns.interface</name><value>default</value></property>
+<property><name>mapred.job.tracker</name><value>local</value></property>
+<property><name>io.seqfile.sorter.recordlimit</name><value>1000000</value></property>
+<property><name>mapred.line.input.format.linespermap</name><value>1</value></property>
+<property><name>mapred.jobtracker.taskScheduler</name><value>org.apache.hadoop.mapred.JobQueueTaskScheduler</value></property>
+<property><name>mapred.tasktracker.instrumentation</name><value>org.apache.hadoop.mapred.TaskTrackerMetricsInst</value></property>
+<property><name>mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill</name><value>5000</value></property>
+<property><name>mapred.local.dir.minspacekill</name><value>0</value></property>
+<property><name>io.sort.record.percent</name><value>0.05</value></property>
+<property><name>fs.kfs.impl</name><value>org.apache.hadoop.fs.kfs.KosmosFileSystem</value></property>
+<property><name>mapred.temp.dir</name><value>${hadoop.tmp.dir}/mapred/temp</value></property>
+<property><name>mapred.tasktracker.reduce.tasks.maximum</name><value>2</value></property>
+<property><name>fs.checkpoint.edits.dir</name><value>${fs.checkpoint.dir}</value></property>
+<property><name>mapred.job.reduce.input.buffer.percent</name><value>0.0</value></property>
+<property><name>mapred.tasktracker.indexcache.mb</name><value>10</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
+<property><name>hadoop.logfile.count</name><value>10</value></property>
+<property><name>mapred.skip.reduce.auto.incr.proc.count</name><value>true</value></property>
+<property><name>io.seqfile.compress.blocksize</name><value>1000000</value></property>
+<property><name>fs.s3.block.size</name><value>67108864</value></property>
+<property><name>mapred.tasktracker.taskmemorymanager.monitoring-interval</name><value>5000</value></property>
 <property><name>mapred.acls.enabled</name><value>false</value></property>
+<property><name>mapred.queue.names</name><value>default</value></property>
+<property><name>fs.hsftp.impl</name><value>org.apache.hadoop.hdfs.HsftpFileSystem</value></property>
+<property><name>mapred.task.tracker.http.address</name><value>0.0.0.0:50060</value></property>
+<property><name>pregelix.vertexClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex</value></property>
+<property><name>mapred.reduce.parallel.copies</name><value>5</value></property>
+<property><name>io.seqfile.lazydecompress</name><value>true</value></property>
+<property><name>mapred.output.dir</name><value>/resultcomplex</value></property>
+<property><name>io.sort.mb</name><value>100</value></property>
+<property><name>ipc.client.connection.maxidletime</name><value>10000</value></property>
+<property><name>mapred.compress.map.output</name><value>false</value></property>
+<property><name>mapred.task.tracker.report.address</name><value>127.0.0.1:0</value></property>
+<property><name>ipc.client.kill.max</name><value>10</value></property>
+<property><name>ipc.client.connect.max.retries</name><value>10</value></property>
+<property><name>fs.s3.impl</name><value>org.apache.hadoop.fs.s3.S3FileSystem</value></property>
+<property><name>mapred.job.tracker.http.address</name><value>0.0.0.0:50030</value></property>
+<property><name>mapred.input.dir</name><value>file:/webmapcomplex</value></property>
+<property><name>io.file.buffer.size</name><value>4096</value></property>
+<property><name>mapred.jobtracker.restart.recover</name><value>false</value></property>
+<property><name>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
+<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat</value></property>
+<property><name>mapred.task.profile</name><value>false</value></property>
+<property><name>jobclient.output.filter</name><value>FAILED</value></property>
+<property><name>mapred.tasktracker.map.tasks.maximum</name><value>2</value></property>
+<property><name>io.compression.codecs</name><value>org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec</value></property>
+<property><name>fs.checkpoint.size</name><value>67108864</value></property>
 </configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
index 636b055..8969fd7 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
@@ -127,6 +127,7 @@
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat</value></property>
 <property><name>mapred.job.queue.name</name><value>default</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>false</value></property>
 <property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
 <property><name>topology.script.number.args</name><value>100</value></property>
 <property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index 6564eb0..245c0f5 100644
--- a/pregelix/pregelix-runtime/pom.xml
+++ b/pregelix/pregelix-runtime/pom.xml
@@ -141,6 +141,13 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-common</artifactId>
+			<version>0.2.10-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-storage-am-btree</artifactId>
 			<version>0.2.10-SNAPSHOT</version>
 			<type>jar</type>
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 f3a0bb4..a002be0 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
@@ -31,6 +31,8 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.graph.Vertex;
@@ -110,7 +112,8 @@
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
                     throws HyracksDataException {
                 this.conf = confFactory.createConfiguration(ctx);
-                this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf);
+                //LSM index does not have in-place update
+                this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf) || BspUtils.useLSM(conf);
                 this.aggregator = BspUtils.createGlobalAggregator(conf);
                 this.aggregator.init();
 
@@ -197,6 +200,10 @@
                     }
                     vertex.finishCompute();
                 } catch (Exception e) {
+                    ClassLoader cl1 = vertex.getClass().getClassLoader();
+                    ClassLoader cl2 = msgContentList.get(0).getClass().getClassLoader();
+                    System.out.println("cl1 " + cl1);
+                    System.out.println("cl2 " + cl2);
                     throw new HyracksDataException(e);
                 }
 
@@ -262,7 +269,8 @@
             }
 
             @Override
-            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor)
+                    throws HyracksDataException {
                 try {
                     if (vertex != null && vertex.hasUpdate()) {
                         if (!dynamicStateLength) {
@@ -271,12 +279,13 @@
                             int offset = tupleRef.getFieldStart(1);
                             bbos.setByteArray(data, offset);
                             vertex.write(output);
+                            ITreeIndexCursor tCursor = (ITreeIndexCursor) cursor;
+                            tCursor.markCurrentTupleAsUpdated();
                         } else {
                             // write the vertex id
                             DataOutput tbOutput = cloneUpdateTb.getDataOutput();
                             vertex.getVertexId().write(tbOutput);
                             cloneUpdateTb.addFieldEndOffset();
-
                             // write the vertex value
                             vertex.write(tbOutput);
                             cloneUpdateTb.addFieldEndOffset();
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java
index 88577c2..8947c01 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
 
@@ -57,7 +58,8 @@
             }
 
             @Override
-            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor)
+                    throws HyracksDataException {
 
             }
 
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 ca8ec01..1d26c83 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
@@ -31,6 +31,8 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
 import edu.uci.ics.pregelix.api.graph.MsgList;
 import edu.uci.ics.pregelix.api.graph.Vertex;
@@ -113,7 +115,8 @@
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
                     throws HyracksDataException {
                 this.conf = confFactory.createConfiguration(ctx);
-                this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf);
+                //LSM index does not have in-place update
+                this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf) || BspUtils.useLSM(conf);;
                 this.aggregator = BspUtils.createGlobalAggregator(conf);
                 this.aggregator.init();
 
@@ -255,7 +258,8 @@
             }
 
             @Override
-            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException {
+            public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor)
+                    throws HyracksDataException {
                 try {
                     if (vertex != null && vertex.hasUpdate()) {
                         if (!dynamicStateLength) {
@@ -264,6 +268,8 @@
                             int offset = tupleRef.getFieldStart(1);
                             bbos.setByteArray(data, offset);
                             vertex.write(output);
+                            ITreeIndexCursor tCursor = (ITreeIndexCursor) cursor;
+                            tCursor.markCurrentTupleAsUpdated();
                         } else {
                             // write the vertex id
                             DataOutput tbOutput = cloneUpdateTb.getDataOutput();
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java
index acd766e..d243c8a 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java
@@ -99,7 +99,7 @@
             }
 
             @Override
-            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 Pair<ArrayBackedValueStorage[], IAggregateFunction[]> aggState = (Pair<ArrayBackedValueStorage[], IAggregateFunction[]>) state.state;
                 ArrayBackedValueStorage[] aggOutput = aggState.getLeft();
@@ -113,6 +113,7 @@
                         throw new HyracksDataException(e);
                     }
                 }
+                return true;
             }
 
             @Override
@@ -121,7 +122,7 @@
             }
 
             @Override
-            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 throw new IllegalStateException("this method should not be called");
             }
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java
index b121b5b..e99fcb3 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java
@@ -17,32 +17,30 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.JobConf;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.util.ArrayListWritable;
 
-@SuppressWarnings("deprecation")
 public class DatatypeHelper {
     private static final class WritableSerializerDeserializer<T extends Writable> implements ISerializerDeserializer<T> {
         private static final long serialVersionUID = 1L;
 
-        private Class<T> clazz;
+        private final Class<T> clazz;
+        private transient Configuration conf;
         private T object;
 
-        private WritableSerializerDeserializer(Class<T> clazz) {
+        private WritableSerializerDeserializer(Class<T> clazz, Configuration conf) {
             this.clazz = clazz;
+            this.conf = conf;
         }
 
-        @SuppressWarnings("unchecked")
+        @SuppressWarnings({ "unchecked", "rawtypes" })
         private T createInstance() throws HyracksDataException {
             // TODO remove "if", create a new WritableInstanceOperations class
             // that deals with Writables that don't have public constructors
@@ -50,7 +48,11 @@
                 return (T) NullWritable.get();
             }
             try {
-                return clazz.newInstance();
+                T t = clazz.newInstance();
+                if (t instanceof ArrayListWritable) {
+                    ((ArrayListWritable) t).setConf(conf);
+                }
+                return t;
             } catch (InstantiationException e) {
                 throw new HyracksDataException(e);
             } catch (IllegalAccessException e) {
@@ -85,42 +87,16 @@
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
     public static ISerializerDeserializer<? extends Writable> createSerializerDeserializer(
-            Class<? extends Writable> fClass) {
-        return new WritableSerializerDeserializer(fClass);
+            Class<? extends Writable> fClass, Configuration conf) {
+        return new WritableSerializerDeserializer(fClass, conf);
     }
 
     public static RecordDescriptor createKeyValueRecordDescriptor(Class<? extends Writable> keyClass,
-            Class<? extends Writable> valueClass) {
+            Class<? extends Writable> valueClass, Configuration conf) {
         @SuppressWarnings("rawtypes")
         ISerializerDeserializer[] fields = new ISerializerDeserializer[2];
-        fields[0] = createSerializerDeserializer(keyClass);
-        fields[1] = createSerializerDeserializer(valueClass);
+        fields[0] = createSerializerDeserializer(keyClass, conf);
+        fields[1] = createSerializerDeserializer(valueClass, conf);
         return new RecordDescriptor(fields);
     }
-
-    public static RecordDescriptor createOneFieldRecordDescriptor(Class<? extends Writable> fieldClass) {
-        @SuppressWarnings("rawtypes")
-        ISerializerDeserializer[] fields = new ISerializerDeserializer[1];
-        fields[0] = createSerializerDeserializer(fieldClass);
-        return new RecordDescriptor(fields);
-    }
-
-    public static JobConf map2JobConf(Map<String, String> jobConfMap) {
-        JobConf jobConf;
-        synchronized (Configuration.class) {
-            jobConf = new JobConf();
-            for (Entry<String, String> entry : jobConfMap.entrySet()) {
-                jobConf.set(entry.getKey(), entry.getValue());
-            }
-        }
-        return jobConf;
-    }
-
-    public static Map<String, String> jobConf2Map(JobConf jobConf) {
-        Map<String, String> jobConfMap = new HashMap<String, String>();
-        for (Entry<String, String> entry : jobConf) {
-            jobConfMap.put(entry.getKey(), entry.getValue());
-        }
-        return jobConfMap;
-    }
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
index 5e8ac1e..3151df2 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.pregelix.runtime.touchpoint;
 
+import java.lang.reflect.Field;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -21,8 +23,6 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.hdfs.ContextFactory;
-import edu.uci.ics.pregelix.api.graph.Vertex;
-import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHook;
 import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
@@ -48,8 +48,12 @@
                 try {
                     TaskAttemptContext mapperContext = ctxFactory.createContext(conf, new TaskAttemptID());
                     mapperContext.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
-                    Vertex.setContext(mapperContext);
-                    BspUtils.setDefaultConfiguration(conf);
+
+                    ClassLoader cl = ctx.getJobletContext().getClassLoader();
+                    Class<?> vClass = (Class<?>) cl.loadClass("edu.uci.ics.pregelix.api.graph.Vertex");
+                    Field contextField = vClass.getDeclaredField("context");
+                    contextField.setAccessible(true);
+                    contextField.set(null, mapperContext);
                 } catch (Exception e) {
                     throw new HyracksDataException(e);
                 }
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java
index 6fa6434..c9b67fb 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java
@@ -16,6 +16,7 @@
 
 import java.io.DataInputStream;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
@@ -24,30 +25,39 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.ISerializerDeserializerFactory;
 
 public class VertexIdPartitionComputerFactory<K extends Writable, V extends Writable> implements
         ITuplePartitionComputerFactory {
     private static final long serialVersionUID = 1L;
     private final ISerializerDeserializerFactory<K> keyIOFactory;
+    private final IConfigurationFactory confFactory;
 
-    public VertexIdPartitionComputerFactory(ISerializerDeserializerFactory<K> keyIOFactory) {
+    public VertexIdPartitionComputerFactory(ISerializerDeserializerFactory<K> keyIOFactory,
+            IConfigurationFactory confFactory) {
         this.keyIOFactory = keyIOFactory;
+        this.confFactory = confFactory;
     }
 
     public ITuplePartitionComputer createPartitioner() {
-        return new ITuplePartitionComputer() {
-            private final ByteBufferInputStream bbis = new ByteBufferInputStream();
-            private final DataInputStream dis = new DataInputStream(bbis);
-            private final ISerializerDeserializer<K> keyIO = keyIOFactory.getSerializerDeserializer();
+        try {
+            final Configuration conf = confFactory.createConfiguration();
+            return new ITuplePartitionComputer() {
+                private final ByteBufferInputStream bbis = new ByteBufferInputStream();
+                private final DataInputStream dis = new DataInputStream(bbis);
+                private final ISerializerDeserializer<K> keyIO = keyIOFactory.getSerializerDeserializer(conf);
 
-            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
-                int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
-                        + accessor.getFieldStartOffset(tIndex, 0);
-                bbis.setByteBuffer(accessor.getBuffer(), keyStart);
-                K key = keyIO.deserialize(dis);
-                return Math.abs(key.hashCode() % nParts);
-            }
-        };
+                public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+                    int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+                            + accessor.getFieldStartOffset(tIndex, 0);
+                    bbis.setByteBuffer(accessor.getBuffer(), keyStart);
+                    K key = keyIO.deserialize(dis);
+                    return Math.abs(key.hashCode() % nParts);
+                }
+            };
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
     }
 }
\ No newline at end of file
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java
index 435d081..c11ac5b 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.pregelix.runtime.touchpoint;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -29,7 +30,7 @@
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
-    public ISerializerDeserializer getSerializerDeserializer() {
-        return DatatypeHelper.createSerializerDeserializer(clazz);
+    public ISerializerDeserializer getSerializerDeserializer(Configuration conf) {
+        return DatatypeHelper.createSerializerDeserializer(clazz, conf);
     }
 }