Several major changes in hyracks:
-- reduced CC/NC communications for reporting partition request and availability; partition request/availability are only reported for the case of send-side materialized (without pipelining) policies in case of task re-attempt.
-- changed buffer cache to dynamically allocate memory based on needs instead of pre-allocating
-- changed each network channel to lazily allocate memory based on needs, and changed materialized connectors to lazily allocate files based on needs
-- changed several major CCNCCFunctions to use non-java serde
-- added a sort-based group-by operator which pushes group-by aggregations into an external sort
-- make external sort a stable sort

1,3,and 4 is to reduce the job overhead.
2 is to reduce the unecessary NC resource consumptions such as memory and files.
5 and 6 are improvements to runtime operators.

One change in algebricks:
-- implemented a rule to push group-by aggregation into sort, i.e., using the sort-based gby operator

Several important changes in pregelix:
-- remove static states in vertex
-- direct check halt bit without deserialization
-- optimize the sort algorithm by packing yet-another 2-byte normalized key into the tPointers array

Change-Id: Id696f9a9f1647b4a025b8b33d20b3a89127c60d6
Reviewed-on: http://fulliautomatix.ics.uci.edu:8443/35
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <westmann@gmail.com>
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index c9ef2f3..f6de971 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -41,6 +41,7 @@
     RANDOM_MERGE_EXCHANGE,
     RTREE_SEARCH,
     RUNNING_AGGREGATE,
+    SORT_GROUP_BY,
     SORT_MERGE_EXCHANGE,
     SINK,
     SINK_WRITE,
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IMergeAggregationExpressionFactory.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IMergeAggregationExpressionFactory.java
index 6e366ff..6a6989d 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IMergeAggregationExpressionFactory.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/IMergeAggregationExpressionFactory.java
@@ -17,8 +17,9 @@
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 
 public interface IMergeAggregationExpressionFactory {
-    ILogicalExpression createMergeAggregation(ILogicalExpression expr, IOptimizationContext env)
+    ILogicalExpression createMergeAggregation(LogicalVariable originalAggVariable, ILogicalExpression expr, IOptimizationContext env)
             throws AlgebricksException;
 }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java
index 1ddfe64..4241146 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/expressions/LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java
@@ -136,10 +136,17 @@
                 }
 
                 @Override
+                public void finishPartial(IPointable result) throws AlgebricksException {
+                    caf.finishPartial();
+                    result.set(abvs);
+                }
+
+                @Override
                 public void finish(IPointable result) throws AlgebricksException {
                     caf.finish();
                     result.set(abvs);
                 }
+
             };
         }
     }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
new file mode 100644
index 0000000..e92e3ee
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortGroupByPOperator.java
@@ -0,0 +1,272 @@
+/*
+ * 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.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.ListSet;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IPartialAggregationTypeComputer;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.OperatorSchemaImpl;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.sort.SortGroupByOperatorDescriptor;
+
+public class SortGroupByPOperator extends AbstractPhysicalOperator {
+
+    private final int frameLimit;
+    private final OrderColumn[] orderColumns;
+    private final List<LogicalVariable> columnSet = new ArrayList<LogicalVariable>();
+
+    public SortGroupByPOperator(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList, int frameLimit,
+            OrderColumn[] orderColumns) {
+        this.frameLimit = frameLimit;
+        this.orderColumns = orderColumns;
+        computeColumnSet(gbyList);
+    }
+
+    private void computeColumnSet(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList) {
+        columnSet.clear();
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gbyList) {
+            ILogicalExpression expr = p.second.getValue();
+            if (expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression v = (VariableReferenceExpression) expr;
+                columnSet.add(v.getVariableReference());
+            }
+        }
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.SORT_GROUP_BY;
+    }
+
+    @Override
+    public String toString() {
+        return getOperatorTag().toString() + columnSet;
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    private List<LogicalVariable> getGbyColumns() {
+        return columnSet;
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+        List<ILocalStructuralProperty> propsLocal = new LinkedList<ILocalStructuralProperty>();
+
+        GroupByOperator gOp = (GroupByOperator) op;
+        Set<LogicalVariable> columnSet = new ListSet<LogicalVariable>();
+
+        if (!columnSet.isEmpty()) {
+            propsLocal.add(new LocalGroupingProperty(columnSet));
+        }
+        for (OrderColumn oc : orderColumns) {
+            propsLocal.add(new LocalOrderProperty(oc));
+        }
+        for (ILogicalPlan p : gOp.getNestedPlans()) {
+            for (Mutable<ILogicalOperator> r : p.getRoots()) {
+                ILogicalOperator rOp = r.getValue();
+                propsLocal.addAll(rOp.getDeliveredPhysicalProperties().getLocalProperties());
+            }
+        }
+
+        ILogicalOperator op2 = op.getInputs().get(0).getValue();
+        IPhysicalPropertiesVector childProp = op2.getDeliveredPhysicalProperties();
+        deliveredProperties = new StructuralPropertiesVector(childProp.getPartitioningProperty(), propsLocal);
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent) {
+        return emptyUnaryRequirements();
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        List<LogicalVariable> gbyCols = getGbyColumns();
+        int keys[] = JobGenHelper.variablesToFieldIndexes(gbyCols, inputSchemas[0]);
+        GroupByOperator gby = (GroupByOperator) op;
+        int numFds = gby.getDecorList().size();
+        int fdColumns[] = new int[numFds];
+        int j = 0;
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gby.getDecorList()) {
+            ILogicalExpression expr = p.second.getValue();
+            if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                throw new AlgebricksException("Sort group-by expects variable references.");
+            }
+            VariableReferenceExpression v = (VariableReferenceExpression) expr;
+            LogicalVariable decor = v.getVariableReference();
+            fdColumns[j++] = inputSchemas[0].findVariable(decor);
+        }
+
+        if (gby.getNestedPlans().size() != 1) {
+            throw new AlgebricksException(
+                    "Sort group-by currently works only for one nested plan with one root containing"
+                            + "an aggregate and a nested-tuple-source.");
+        }
+        ILogicalPlan p0 = gby.getNestedPlans().get(0);
+        if (p0.getRoots().size() != 1) {
+            throw new AlgebricksException(
+                    "Sort group-by currently works only for one nested plan with one root containing"
+                            + "an aggregate and a nested-tuple-source.");
+        }
+        Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
+        AggregateOperator aggOp = (AggregateOperator) r0.getValue();
+
+        IPartialAggregationTypeComputer partialAggregationTypeComputer = context.getPartialAggregationTypeComputer();
+        List<Object> intermediateTypes = new ArrayList<Object>();
+        int n = aggOp.getExpressions().size();
+        IAggregateEvaluatorFactory[] aff = new IAggregateEvaluatorFactory[n];
+        int i = 0;
+        IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
+        IVariableTypeEnvironment aggOpInputEnv = context.getTypeEnvironment(aggOp.getInputs().get(0).getValue());
+        IVariableTypeEnvironment outputEnv = context.getTypeEnvironment(op);
+        for (Mutable<ILogicalExpression> exprRef : aggOp.getExpressions()) {
+            AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) exprRef.getValue();
+            aff[i++] = expressionRuntimeProvider.createAggregateFunctionFactory(aggFun, aggOpInputEnv, inputSchemas,
+                    context);
+            intermediateTypes.add(partialAggregationTypeComputer.getType(aggFun, aggOpInputEnv,
+                    context.getMetadataProvider()));
+        }
+
+        int[] keyAndDecFields = new int[keys.length + fdColumns.length];
+        for (i = 0; i < keys.length; ++i) {
+            keyAndDecFields[i] = keys[i];
+        }
+        for (i = 0; i < fdColumns.length; i++) {
+            keyAndDecFields[keys.length + i] = fdColumns[i];
+        }
+
+        List<LogicalVariable> keyAndDecVariables = new ArrayList<LogicalVariable>();
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gby.getGroupByList())
+            keyAndDecVariables.add(p.first);
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : gby.getDecorList())
+            keyAndDecVariables.add(GroupByOperator.getDecorVariable(p));
+
+        for (LogicalVariable var : keyAndDecVariables)
+            aggOpInputEnv.setVarType(var, outputEnv.getVarType(var));
+
+        compileSubplans(inputSchemas[0], gby, opSchema, context);
+        IOperatorDescriptorRegistry spec = builder.getJobSpec();
+
+        IBinaryComparatorFactory[] compFactories = new IBinaryComparatorFactory[gbyCols.size()];
+        IBinaryComparatorFactoryProvider bcfProvider = context.getBinaryComparatorFactoryProvider();
+        i = 0;
+        for (LogicalVariable v : gbyCols) {
+            Object type = aggOpInputEnv.getVarType(v);
+            if (orderColumns[i].getOrder() == OrderKind.ASC) {
+                compFactories[i] = bcfProvider.getBinaryComparatorFactory(type, true);
+            } else {
+                compFactories[i] = bcfProvider.getBinaryComparatorFactory(type, false);
+            }
+            i++;
+        }
+        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema,
+                context);
+
+        IAggregateEvaluatorFactory[] merges = new IAggregateEvaluatorFactory[n];
+        List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+        IOperatorSchema[] localInputSchemas = new IOperatorSchema[1];
+        localInputSchemas[0] = new OperatorSchemaImpl();
+        for (i = 0; i < n; i++) {
+            AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) aggOp.getMergeExpressions()
+                    .get(i).getValue();
+            aggFun.getUsedVariables(usedVars);
+        }
+        i = 0;
+        for (Object type : intermediateTypes) {
+            aggOpInputEnv.setVarType(usedVars.get(i++), type);
+        }
+        for (LogicalVariable keyVar : keyAndDecVariables)
+            localInputSchemas[0].addVariable(keyVar);
+        for (LogicalVariable usedVar : usedVars)
+            localInputSchemas[0].addVariable(usedVar);
+        for (i = 0; i < n; i++) {
+            AggregateFunctionCallExpression mergeFun = (AggregateFunctionCallExpression) aggOp.getMergeExpressions()
+                    .get(i).getValue();
+            merges[i] = expressionRuntimeProvider.createAggregateFunctionFactory(mergeFun, aggOpInputEnv,
+                    localInputSchemas, context);
+        }
+        RecordDescriptor partialAggRecordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
+                localInputSchemas[0], context);
+
+        IAggregatorDescriptorFactory aggregatorFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(aff,
+                keyAndDecFields);
+        IAggregatorDescriptorFactory mergeFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(merges,
+                keyAndDecFields);
+
+        INormalizedKeyComputerFactory normalizedKeyFactory = null;
+        INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+        if (nkcfProvider == null) {
+            normalizedKeyFactory = null;
+        }
+        Object type = aggOpInputEnv.getVarType(gbyCols.get(0));
+        normalizedKeyFactory = orderColumns[0].getOrder() == OrderKind.ASC ? nkcfProvider
+                .getNormalizedKeyComputerFactory(type, true) : nkcfProvider
+                .getNormalizedKeyComputerFactory(type, false);
+        SortGroupByOperatorDescriptor gbyOpDesc = new SortGroupByOperatorDescriptor(spec, frameLimit, keys,
+                keyAndDecFields, normalizedKeyFactory, compFactories, aggregatorFactory, mergeFactory,
+                partialAggRecordDescriptor, recordDescriptor, false);
+
+        contributeOpDesc(builder, gby, gbyOpDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
+    }
+}
diff --git a/algebricks/algebricks-examples/piglet-example/pom.xml b/algebricks/algebricks-examples/piglet-example/pom.xml
index a343d98..f32689f 100644
--- a/algebricks/algebricks-examples/piglet-example/pom.xml
+++ b/algebricks/algebricks-examples/piglet-example/pom.xml
@@ -16,8 +16,7 @@
   <modelVersion>4.0.0</modelVersion>
   <artifactId>piglet-example</artifactId>
   <name>piglet-example</name>
-
-  <parent>
+ <parent>
     <groupId>edu.uci.ics.hyracks</groupId>
     <artifactId>algebricks-examples</artifactId>
     <version>0.2.12-SNAPSHOT</version>
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushGroupByIntoSortRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushGroupByIntoSortRule.java
new file mode 100644
index 0000000..56b2a8e
--- /dev/null
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushGroupByIntoSortRule.java
@@ -0,0 +1,150 @@
+/*
+ * 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.rewriter.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+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.IMergeAggregationExpressionFactory;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractStableSortPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.SortGroupByPOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * @author yingyib
+ *         merge externalsort+preclustered-gby into sort-gby
+ */
+public class PushGroupByIntoSortRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator op1 = opRef.getValue();
+        if (op1 == null) {
+            return false;
+        }
+        boolean changed = false;
+        for (Mutable<ILogicalOperator> childRef : op1.getInputs()) {
+            AbstractLogicalOperator op = (AbstractLogicalOperator) childRef.getValue();
+            if (op.getOperatorTag() == LogicalOperatorTag.GROUP) {
+                PhysicalOperatorTag opTag = op.getPhysicalOperator().getOperatorTag();
+                GroupByOperator groupByOperator = (GroupByOperator) op;
+                if (opTag == PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY) {
+                    Mutable<ILogicalOperator> op2Ref = op.getInputs().get(0).getValue().getInputs().get(0);
+                    AbstractLogicalOperator op2 = (AbstractLogicalOperator) op2Ref.getValue();
+                    if (op2.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.STABLE_SORT) {
+                        AbstractStableSortPOperator sortPhysicalOperator = (AbstractStableSortPOperator) op2
+                                .getPhysicalOperator();
+                        if (groupByOperator.getNestedPlans().size() != 1) {
+                            //Sort group-by currently works only for one nested plan with one root containing
+                            //an aggregate and a nested-tuple-source.
+                            continue;
+                        }
+                        ILogicalPlan p0 = groupByOperator.getNestedPlans().get(0);
+                        if (p0.getRoots().size() != 1) {
+                            //Sort group-by currently works only for one nested plan with one root containing
+                            //an aggregate and a nested-tuple-source.
+                            continue;
+                        }
+
+                        Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
+                        AbstractLogicalOperator r0Logical = (AbstractLogicalOperator) r0.getValue();
+                        if (r0Logical.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+                            //we only rewrite aggregation function; do nothing for running aggregates
+                            continue;
+                        }
+                        AggregateOperator aggOp = (AggregateOperator) r0.getValue();
+                        AbstractLogicalOperator aggInputOp = (AbstractLogicalOperator) aggOp.getInputs().get(0)
+                                .getValue();
+                        if (aggInputOp.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+                            continue;
+                        }
+
+                        boolean hasIntermediateAggregate = generateMergeAggregationExpressions(groupByOperator, context);
+                        if (!hasIntermediateAggregate) {
+                            continue;
+                        }
+
+                        //replace preclustered gby with sort gby
+                        op.setPhysicalOperator(new SortGroupByPOperator(groupByOperator.getGroupByList(), context
+                                .getPhysicalOptimizationConfig().getMaxFramesExternalGroupBy(), sortPhysicalOperator
+                                .getSortColumns()));
+
+                        // remove the stable sort operator
+                        op.getInputs().clear();
+                        op.getInputs().addAll(op2.getInputs());
+                        changed = true;
+                    }
+                }
+                continue;
+            } else {
+                continue;
+            }
+        }
+        return changed;
+    }
+
+    private boolean generateMergeAggregationExpressions(GroupByOperator gby, IOptimizationContext context)
+            throws AlgebricksException {
+        if (gby.getNestedPlans().size() != 1) {
+            throw new AlgebricksException(
+                    "External/sort group-by currently works only for one nested plan with one root containing"
+                            + "an aggregate and a nested-tuple-source.");
+        }
+        ILogicalPlan p0 = gby.getNestedPlans().get(0);
+        if (p0.getRoots().size() != 1) {
+            throw new AlgebricksException(
+                    "External/sort group-by currently works only for one nested plan with one root containing"
+                            + "an aggregate and a nested-tuple-source.");
+        }
+        IMergeAggregationExpressionFactory mergeAggregationExpressionFactory = context
+                .getMergeAggregationExpressionFactory();
+        Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
+        AggregateOperator aggOp = (AggregateOperator) r0.getValue();
+        List<Mutable<ILogicalExpression>> aggFuncRefs = aggOp.getExpressions();
+        List<LogicalVariable> originalAggVars = aggOp.getVariables();
+        int n = aggOp.getExpressions().size();
+        List<Mutable<ILogicalExpression>> mergeExpressionRefs = new ArrayList<Mutable<ILogicalExpression>>();
+        for (int i = 0; i < n; i++) {
+            ILogicalExpression mergeExpr = mergeAggregationExpressionFactory.createMergeAggregation(
+                    originalAggVars.get(i), aggFuncRefs.get(i).getValue(), context);
+            if (mergeExpr == null) {
+                return false;
+            }
+            mergeExpressionRefs.add(new MutableObject<ILogicalExpression>(mergeExpr));
+        }
+        aggOp.setMergeExpressions(mergeExpressionRefs);
+        return true;
+    }
+}
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 9efb078..fd7d31d 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 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.OperatorAnnotations;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory;
@@ -44,7 +45,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
@@ -150,12 +150,16 @@
                                     throw new NotImplementedException(
                                             "External hash group-by for nested grouping is not implemented.");
                                 }
-                                ExternalGroupByPOperator externalGby = new ExternalGroupByPOperator(
-                                        gby.getGroupByList(), physicalOptimizationConfig.getMaxFramesExternalGroupBy(),
-                                        physicalOptimizationConfig.getExternalGroupByTableSize());
-                                op.setPhysicalOperator(externalGby);
-                                generateMergeAggregationExpressions(gby, context);
-                                break;
+
+                                boolean hasIntermediateAgg = generateMergeAggregationExpressions(gby, context);
+                                if (hasIntermediateAgg) {
+                                    ExternalGroupByPOperator externalGby = new ExternalGroupByPOperator(
+                                            gby.getGroupByList(),
+                                            physicalOptimizationConfig.getMaxFramesExternalGroupBy(),
+                                            physicalOptimizationConfig.getExternalGroupByTableSize());
+                                    op.setPhysicalOperator(externalGby);
+                                    break;
+                                }
                             }
                         }
                     }
@@ -331,15 +335,19 @@
         return payload;
     }
 
-    private static void generateMergeAggregationExpressions(GroupByOperator gby, IOptimizationContext context)
+    private static boolean generateMergeAggregationExpressions(GroupByOperator gby, IOptimizationContext context)
             throws AlgebricksException {
         if (gby.getNestedPlans().size() != 1) {
+            //External/Sort group-by currently works only for one nested plan with one root containing
+            //an aggregate and a nested-tuple-source.
             throw new AlgebricksException(
                     "External group-by currently works only for one nested plan with one root containing"
                             + "an aggregate and a nested-tuple-source.");
         }
         ILogicalPlan p0 = gby.getNestedPlans().get(0);
         if (p0.getRoots().size() != 1) {
+            //External/Sort group-by currently works only for one nested plan with one root containing
+            //an aggregate and a nested-tuple-source.
             throw new AlgebricksException(
                     "External group-by currently works only for one nested plan with one root containing"
                             + "an aggregate and a nested-tuple-source.");
@@ -347,15 +355,24 @@
         IMergeAggregationExpressionFactory mergeAggregationExpressionFactory = context
                 .getMergeAggregationExpressionFactory();
         Mutable<ILogicalOperator> r0 = p0.getRoots().get(0);
+        AbstractLogicalOperator r0Logical = (AbstractLogicalOperator) r0.getValue();
+        if (r0Logical.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+            return false;
+        }
         AggregateOperator aggOp = (AggregateOperator) r0.getValue();
         List<Mutable<ILogicalExpression>> aggFuncRefs = aggOp.getExpressions();
+        List<LogicalVariable> originalAggVars = aggOp.getVariables();
         int n = aggOp.getExpressions().size();
         List<Mutable<ILogicalExpression>> mergeExpressionRefs = new ArrayList<Mutable<ILogicalExpression>>();
         for (int i = 0; i < n; i++) {
-            ILogicalExpression mergeExpr = mergeAggregationExpressionFactory.createMergeAggregation(aggFuncRefs.get(i)
-                    .getValue(), context);
+            ILogicalExpression mergeExpr = mergeAggregationExpressionFactory.createMergeAggregation(
+                    originalAggVars.get(i), aggFuncRefs.get(i).getValue(), context);
+            if (mergeExpr == null) {
+                return false;
+            }
             mergeExpressionRefs.add(new MutableObject<ILogicalExpression>(mergeExpr));
         }
         aggOp.setMergeExpressions(mergeExpressionRefs);
+        return true;
     }
 }
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java
index 594514c..e6baffa 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/aggregators/TupleCountAggregateFunctionFactory.java
@@ -55,6 +55,11 @@
                     throw new AlgebricksException(e);
                 }
             }
+
+            @Override
+            public void finishPartial(IPointable result) throws AlgebricksException {
+                finish(result);
+            }
         };
     }
 
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IAggregateEvaluator.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IAggregateEvaluator.java
index 99d7b91..5719bff 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IAggregateEvaluator.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/base/IAggregateEvaluator.java
@@ -23,5 +23,7 @@
 
     public void step(IFrameTupleReference tuple) throws AlgebricksException;
 
+    public void finishPartial(IPointable result) throws AlgebricksException;
+
     public void finish(IPointable result) throws AlgebricksException;
 }
\ No newline at end of file
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 11a7a5c..b3eab7b 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
@@ -121,7 +121,16 @@
             @Override
             public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor,
                     int tIndex, AggregateState state) throws HyracksDataException {
-                throw new IllegalStateException("this method should not be called");
+                IAggregateEvaluator[] agg = (IAggregateEvaluator[]) state.state;
+                for (int i = 0; i < agg.length; i++) {
+                    try {
+                        agg[i].finishPartial(p);
+                        tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
+                    } catch (AlgebricksException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+                return true;
             }
 
             @Override
diff --git a/hivesterix/.gitignore b/hivesterix/.gitignore
new file mode 100644
index 0000000..0d0a286
--- /dev/null
+++ b/hivesterix/.gitignore
@@ -0,0 +1 @@
+optest
diff --git a/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java b/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
index b7bb637..41813ba 100644
--- a/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
+++ b/hivesterix/hivesterix-common/src/main/java/edu/uci/ics/hivesterix/logical/expression/HiveMergeAggregationExpressionFactory.java
@@ -44,8 +44,8 @@
     public static IMergeAggregationExpressionFactory INSTANCE = new HiveMergeAggregationExpressionFactory();
 
     @Override
-    public ILogicalExpression createMergeAggregation(ILogicalExpression expr, IOptimizationContext context)
-            throws AlgebricksException {
+    public ILogicalExpression createMergeAggregation(LogicalVariable inputVar, ILogicalExpression expr,
+            IOptimizationContext context) throws AlgebricksException {
         /**
          * type inference for scalar function
          */
@@ -55,7 +55,6 @@
              * hive aggregation info
              */
             AggregationDesc aggregator = (AggregationDesc) ((HiveFunctionInfo) funcExpr.getFunctionInfo()).getInfo();
-            LogicalVariable inputVar = context.newVar();
             ExprNodeDesc col = new ExprNodeColumnDesc(TypeInfoFactory.voidTypeInfo, inputVar.toString(), null, false);
             ArrayList<ExprNodeDesc> parameters = new ArrayList<ExprNodeDesc>();
             parameters.add(col);
diff --git a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
index 7b88de4..a95edd8 100644
--- a/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
+++ b/hivesterix/hivesterix-dist/src/main/java/edu/uci/ics/hivesterix/runtime/exec/HyracksExecutionEngine.java
@@ -589,6 +589,7 @@
             }
         }
 
+        job.setReportTaskDetails(false);
         long start = System.currentTimeMillis();
         JobId jobId = hcc.startJob(job);
         hcc.waitForCompletion(jobId);
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java
index 570f151..c21d2d6 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestCase.java
@@ -14,7 +14,10 @@
  */
 package edu.uci.ics.hivesterix.test.optimizer;
 
+import java.io.BufferedWriter;
 import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 
@@ -60,6 +63,10 @@
         }
         StringBuilder buf = new StringBuilder();
         readFileToString(resultFile, buf);
+        BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(new FileOutputStream("optest/"
+                + resultFile.getName())));
+        writer.write(sw.toString());
+        writer.close();
         if (!buf.toString().equals(sw.toString())) {
             throw new Exception("Result for " + queryFile + " changed:\n" + sw.toString());
         }
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
index a33ac7e..31268cb 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/optimizer/OptimizerTestSuite.java
@@ -21,6 +21,9 @@
 import java.util.List;
 
 import junit.framework.Test;
+
+import org.apache.commons.io.FileUtils;
+
 import edu.uci.ics.hivesterix.test.base.AbstractTestSuiteClass;
 
 public class OptimizerTestSuite extends AbstractTestSuiteClass {
@@ -41,6 +44,7 @@
         try {
             testSuite.setup();
             testSuite.loadData();
+            FileUtils.forceMkdir(new File("optest"));
         } catch (Exception e) {
             e.printStackTrace();
             throw new IllegalStateException(e.getMessage());
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan
index bda1113..c709f25 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q10_returned_item.plan
@@ -22,16 +22,16 @@
                               nested tuple source
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                          }
-                  -- EXTERNAL_GROUP_BY[$$48, $$49, $$50, $$51, $$52, $$53, $$54]  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$48, $$49, $$50, $$51, $$52, $$53, $$54]  |PARTITIONED|
                     exchange 
-                    -- HASH_PARTITION_EXCHANGE [$$48, $$49, $$50, $$51, $$52, $$53, $$54]  |PARTITIONED|
+                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$48(ASC), $$49(ASC), $$50(ASC), $$51(ASC), $$52(ASC), $$53(ASC), $$54(ASC)] HASH:[$$48, $$49, $$50, $$51, $$52, $$53, $$54]  |PARTITIONED|
                       group by ([$$48 := %0->$$21; $$49 := %0->$$22; $$50 := %0->$$26; $$51 := %0->$$25; $$52 := %0->$$18; $$53 := %0->$$23; $$54 := %0->$$28]) decor ([]) {
                                 aggregate [$$47] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$6, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$7]]]]
                                 -- AGGREGATE  |LOCAL|
                                   nested tuple source
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                              }
-                      -- EXTERNAL_GROUP_BY[$$21, $$22, $$26, $$25, $$18, $$23, $$28]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$21, $$22, $$26, $$25, $$18, $$23, $$28]  |PARTITIONED|
                         exchange 
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           project ([$$21, $$22, $$23, $$25, $$26, $$28, $$18, $$6, $$7])
@@ -99,4 +99,4 @@
                                                     exchange 
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       empty-tuple-source
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q11_important_stock.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q11_important_stock.plan
index 8195ef0..9bb9788 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q11_important_stock.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q11_important_stock.plan
@@ -10,16 +10,16 @@
                   nested tuple source
                   -- NESTED_TUPLE_SOURCE  |LOCAL|
              }
-      -- EXTERNAL_GROUP_BY[$$21]  |PARTITIONED|
+      -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
         exchange 
-        -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+        -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$21(ASC)] HASH:[$$21]  |PARTITIONED|
           group by ([$$21 := %0->$$1]) decor ([]) {
                     aggregate [$$20] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$4, %0->$$3]]]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- EXTERNAL_GROUP_BY[$$1]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$1]  |PARTITIONED|
             exchange 
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               project ([$$1, $$3, $$4])
@@ -76,7 +76,7 @@
                 nested tuple source
                 -- NESTED_TUPLE_SOURCE  |LOCAL|
            }
-    -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+    -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
       exchange 
       -- HASH_PARTITION_EXCHANGE []  |PARTITIONED|
         group by ([]) decor ([]) {
@@ -85,7 +85,7 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
           exchange 
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             data-scan [$$2]<-[$$1, $$2] <- default.q11_part_tmp
@@ -123,4 +123,4 @@
                   exchange 
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     empty-tuple-source
-                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan
index d976bba..a6aa730 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q12_shipping.plan
@@ -6,53 +6,49 @@
     -- ASSIGN  |PARTITIONED|
       exchange 
       -- SORT_MERGE_EXCHANGE [$$26(ASC) ]  |PARTITIONED|
-        order (ASC, %0->$$26) 
-        -- STABLE_SORT [$$26(ASC)]  |PARTITIONED|
+        group by ([$$26 := %0->$$34]) decor ([]) {
+                  aggregate [$$27, $$28] <- [function-call: hive:sum(FINAL), Args:[%0->$$32], function-call: hive:sum(FINAL), Args:[%0->$$33]]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               }
+        -- PRE_CLUSTERED_GROUP_BY[$$34]  |PARTITIONED|
           exchange 
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$26 := %0->$$34]) decor ([]) {
-                      aggregate [$$27, $$28] <- [function-call: hive:sum(FINAL), Args:[%0->$$32], function-call: hive:sum(FINAL), Args:[%0->$$33]]
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$34(ASC)] HASH:[$$34]  |PARTITIONED|
+            group by ([$$34 := %0->$$24]) decor ([]) {
+                      aggregate [$$32, $$33] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen, Args:[function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$6, 1-URGENT], function-call: algebricks:eq, Args:[%0->$$6, 2-HIGH]], 1, 0]], function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen, Args:[function-call: algebricks:and, Args:[function-call: algebricks:neq, Args:[%0->$$6, 1-URGENT], function-call: algebricks:neq, Args:[%0->$$6, 2-HIGH]], 1, 0]]]
                       -- AGGREGATE  |LOCAL|
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$34]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$24]  |PARTITIONED|
               exchange 
-              -- HASH_PARTITION_EXCHANGE [$$34]  |PARTITIONED|
-                group by ([$$34 := %0->$$24]) decor ([]) {
-                          aggregate [$$32, $$33] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen, Args:[function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$6, 1-URGENT], function-call: algebricks:eq, Args:[%0->$$6, 2-HIGH]], 1, 0]], function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen, Args:[function-call: algebricks:and, Args:[function-call: algebricks:neq, Args:[%0->$$6, 1-URGENT], function-call: algebricks:neq, Args:[%0->$$6, 2-HIGH]], 1, 0]]]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       }
-                -- EXTERNAL_GROUP_BY[$$24]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$6, $$24])
+                -- STREAM_PROJECT  |PARTITIONED|
                   exchange 
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$6, $$24])
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    join (function-call: algebricks:eq, Args:[%0->$$10, %0->$$1])
+                    -- HYBRID_HASH_JOIN [$$10][$$1]  |PARTITIONED|
                       exchange 
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        join (function-call: algebricks:eq, Args:[%0->$$10, %0->$$1])
-                        -- HYBRID_HASH_JOIN [$$10][$$1]  |PARTITIONED|
-                          exchange 
-                          -- HASH_PARTITION_EXCHANGE [$$10]  |PARTITIONED|
-                            project ([$$10, $$24])
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$21, %0->$$22], function-call: algebricks:lt, Args:[%0->$$20, %0->$$21], function-call: algebricks:ge, Args:[%0->$$22, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$22, 1995-01-01], function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$24, MAIL], function-call: algebricks:eq, Args:[%0->$$24, SHIP]]])
-                              -- STREAM_SELECT  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$10]  |PARTITIONED|
+                        project ([$$10, $$24])
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          select (function-call: algebricks:and, Args:[function-call: algebricks:lt, Args:[%0->$$21, %0->$$22], function-call: algebricks:lt, Args:[%0->$$20, %0->$$21], function-call: algebricks:ge, Args:[%0->$$22, 1994-01-01], function-call: algebricks:lt, Args:[%0->$$22, 1995-01-01], function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$24, MAIL], function-call: algebricks:eq, Args:[%0->$$24, SHIP]]])
+                          -- STREAM_SELECT  |PARTITIONED|
+                            exchange 
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan [$$21, $$20, $$22, $$24, $$10]<-[$$10, $$11, $$12, $$13, $$14, $$15, $$16, $$17, $$18, $$19, $$20, $$21, $$22, $$23, $$24, $$25] <- default.lineitem
+                              -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange 
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan [$$21, $$20, $$22, $$24, $$10]<-[$$10, $$11, $$12, $$13, $$14, $$15, $$16, $$17, $$18, $$19, $$20, $$21, $$22, $$23, $$24, $$25] <- default.lineitem
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    exchange 
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      empty-tuple-source
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  empty-tuple-source
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      exchange 
+                      -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
+                        data-scan [$$1, $$6]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.orders
+                        -- DATASOURCE_SCAN  |PARTITIONED|
                           exchange 
-                          -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
-                            data-scan [$$1, $$6]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.orders
-                            -- DATASOURCE_SCAN  |PARTITIONED|
-                              exchange 
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            empty-tuple-source
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan
index 40cedd6..690b312 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q13_customer_distribution.plan
@@ -16,16 +16,16 @@
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$26]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$26]  |PARTITIONED|
               exchange 
-              -- HASH_PARTITION_EXCHANGE [$$26]  |PARTITIONED|
+              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$26(ASC)] HASH:[$$26]  |PARTITIONED|
                 group by ([$$26 := %0->$$19]) decor ([]) {
                           aggregate [$$25] <- [function-call: hive:count(PARTIAL1), Args:[1]]
                           -- AGGREGATE  |LOCAL|
                             nested tuple source
                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                        }
-                -- EXTERNAL_GROUP_BY[$$19]  |PARTITIONED|
+                -- SORT_GROUP_BY[$$19]  |PARTITIONED|
                   exchange 
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     project ([$$19])
@@ -38,7 +38,7 @@
                                     nested tuple source
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                }
-                        -- EXTERNAL_GROUP_BY[$$10]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$10]  |PARTITIONED|
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             project ([$$10, $$1])
@@ -68,4 +68,4 @@
                                             exchange 
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               empty-tuple-source
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan
index e6d1c1d..fbdb1de 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q14_promotion_effect.plan
@@ -12,7 +12,7 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
           exchange 
           -- HASH_PARTITION_EXCHANGE []  |PARTITIONED|
             group by ([]) decor ([]) {
@@ -21,7 +21,7 @@
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
               exchange 
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 project ([$$5, $$15, $$16])
@@ -51,4 +51,4 @@
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             empty-tuple-source
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan
index c61ed37..2f7aeb3 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q15_top_supplier.plan
@@ -10,16 +10,16 @@
                   nested tuple source
                   -- NESTED_TUPLE_SOURCE  |LOCAL|
              }
-      -- EXTERNAL_GROUP_BY[$$21]  |PARTITIONED|
+      -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
         exchange 
-        -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+        -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$21(ASC)] HASH:[$$21]  |PARTITIONED|
           group by ([$$21 := %0->$$3]) decor ([]) {
                     aggregate [$$20] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$6, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$7]]]]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- EXTERNAL_GROUP_BY[$$3]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$3]  |PARTITIONED|
             exchange 
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               project ([$$3, $$6, $$7])
@@ -107,4 +107,4 @@
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             empty-tuple-source
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan
index c986cb4..1839e9b 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q16_parts_supplier_relationship.plan
@@ -74,16 +74,16 @@
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$5, $$6, $$7]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[$$5, $$6, $$7]  |PARTITIONED|
               exchange 
-              -- HASH_PARTITION_EXCHANGE [$$5, $$6, $$7]  |PARTITIONED|
+              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$5(ASC), $$6(ASC), $$7(ASC), $$8(ASC)] HASH:[$$5, $$6, $$7]  |PARTITIONED|
                 group by ([$$5 := %0->$$1; $$6 := %0->$$2; $$7 := %0->$$3; $$8 := %0->$$4]) decor ([]) {
                           aggregate [] <- []
                           -- AGGREGATE  |LOCAL|
                             nested tuple source
                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                        }
-                -- EXTERNAL_GROUP_BY[$$1, $$2, $$3, $$4]  |PARTITIONED|
+                -- SORT_GROUP_BY[$$1, $$2, $$3, $$4]  |PARTITIONED|
                   exchange 
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     select (function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:or, Args:[function-call: algebricks:eq, Args:[%0->$$3, 49], function-call: algebricks:eq, Args:[%0->$$3, 14]], function-call: algebricks:eq, Args:[%0->$$3, 23]], function-call: algebricks:eq, Args:[%0->$$3, 45]], function-call: algebricks:eq, Args:[%0->$$3, 19]], function-call: algebricks:eq, Args:[%0->$$3, 3]], function-call: algebricks:eq, Args:[%0->$$3, 36]], function-call: algebricks:eq, Args:[%0->$$3, 9]])
@@ -95,4 +95,4 @@
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             empty-tuple-source
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan
index 8a75f64..23b98c7 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q17_small_quantity_order_revenue.plan
@@ -12,16 +12,16 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[$$22]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$22]  |PARTITIONED|
           exchange 
-          -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$22(ASC)] HASH:[$$22]  |PARTITIONED|
             group by ([$$22 := %0->$$2]) decor ([]) {
                       aggregate [$$21] <- [function-call: hive:avg(PARTIAL1), Args:[%0->$$5]]
                       -- AGGREGATE  |LOCAL|
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$2]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$2]  |PARTITIONED|
               exchange 
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 data-scan [$$2, $$5]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
@@ -44,7 +44,7 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
           exchange 
           -- HASH_PARTITION_EXCHANGE []  |PARTITIONED|
             group by ([]) decor ([]) {
@@ -53,7 +53,7 @@
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+            -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
               exchange 
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 project ([$$17])
@@ -101,4 +101,4 @@
                             exchange 
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               empty-tuple-source
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan
index eb78f1d..2aae626 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q18_large_volume_customer.plan
@@ -10,16 +10,16 @@
                   nested tuple source
                   -- NESTED_TUPLE_SOURCE  |LOCAL|
              }
-      -- EXTERNAL_GROUP_BY[$$21]  |PARTITIONED|
+      -- PRE_CLUSTERED_GROUP_BY[$$21]  |PARTITIONED|
         exchange 
-        -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+        -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$21(ASC)] HASH:[$$21]  |PARTITIONED|
           group by ([$$21 := %0->$$1]) decor ([]) {
                     aggregate [$$20] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$5]]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- EXTERNAL_GROUP_BY[$$1]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$1]  |PARTITIONED|
             exchange 
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               data-scan [$$1, $$5]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
@@ -52,16 +52,16 @@
                               nested tuple source
                               -- NESTED_TUPLE_SOURCE  |LOCAL|
                          }
-                  -- EXTERNAL_GROUP_BY[$$44, $$45, $$46, $$47, $$48]  |PARTITIONED|
+                  -- PRE_CLUSTERED_GROUP_BY[$$44, $$45, $$46, $$47, $$48]  |PARTITIONED|
                     exchange 
-                    -- HASH_PARTITION_EXCHANGE [$$44, $$45, $$46, $$47, $$48]  |PARTITIONED|
+                    -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$44(ASC), $$45(ASC), $$46(ASC), $$47(ASC), $$48(ASC)] HASH:[$$44, $$45, $$46, $$47, $$48]  |PARTITIONED|
                       group by ([$$44 := %0->$$20; $$45 := %0->$$19; $$46 := %0->$$27; $$47 := %0->$$31; $$48 := %0->$$30]) decor ([]) {
                                 aggregate [$$43] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$7]]
                                 -- AGGREGATE  |LOCAL|
                                   nested tuple source
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                              }
-                      -- EXTERNAL_GROUP_BY[$$20, $$19, $$27, $$31, $$30]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$20, $$19, $$27, $$31, $$30]  |PARTITIONED|
                         exchange 
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           project ([$$19, $$20, $$27, $$30, $$31, $$7])
@@ -123,4 +123,4 @@
                                             exchange 
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               empty-tuple-source
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan
index 9e97b7a..839f09a 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q19_discounted_revenue.plan
@@ -8,7 +8,7 @@
                 nested tuple source
                 -- NESTED_TUPLE_SOURCE  |LOCAL|
            }
-    -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+    -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
       exchange 
       -- HASH_PARTITION_EXCHANGE []  |PARTITIONED|
         group by ([]) decor ([]) {
@@ -17,7 +17,7 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
           exchange 
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             project ([$$15, $$16])
@@ -43,4 +43,4 @@
                         exchange 
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           empty-tuple-source
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q1_pricing_summary_report.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q1_pricing_summary_report.plan
index de964ac..3505205 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q1_pricing_summary_report.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q1_pricing_summary_report.plan
@@ -6,37 +6,33 @@
     -- ASSIGN  |PARTITIONED|
       exchange 
       -- SORT_MERGE_EXCHANGE [$$17(ASC), $$18(ASC) ]  |PARTITIONED|
-        order (ASC, %0->$$17) (ASC, %0->$$18) 
-        -- STABLE_SORT [$$17(ASC), $$18(ASC)]  |PARTITIONED|
+        group by ([$$17 := %0->$$37; $$18 := %0->$$38]) decor ([]) {
+                  aggregate [$$19, $$20, $$21, $$22, $$23, $$24, $$25, $$26] <- [function-call: hive:sum(FINAL), Args:[%0->$$29], function-call: hive:sum(FINAL), Args:[%0->$$30], function-call: hive:sum(FINAL), Args:[%0->$$31], function-call: hive:sum(FINAL), Args:[%0->$$32], function-call: hive:avg(FINAL), Args:[%0->$$33], function-call: hive:avg(FINAL), Args:[%0->$$34], function-call: hive:avg(FINAL), Args:[%0->$$35], function-call: hive:count(FINAL), Args:[%0->$$36]]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               }
+        -- PRE_CLUSTERED_GROUP_BY[$$37, $$38]  |PARTITIONED|
           exchange 
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$17 := %0->$$37; $$18 := %0->$$38]) decor ([]) {
-                      aggregate [$$19, $$20, $$21, $$22, $$23, $$24, $$25, $$26] <- [function-call: hive:sum(FINAL), Args:[%0->$$29], function-call: hive:sum(FINAL), Args:[%0->$$30], function-call: hive:sum(FINAL), Args:[%0->$$31], function-call: hive:sum(FINAL), Args:[%0->$$32], function-call: hive:avg(FINAL), Args:[%0->$$33], function-call: hive:avg(FINAL), Args:[%0->$$34], function-call: hive:avg(FINAL), Args:[%0->$$35], function-call: hive:count(FINAL), Args:[%0->$$36]]
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$37(ASC), $$38(ASC)] HASH:[$$37, $$38]  |PARTITIONED|
+            group by ([$$37 := %0->$$9; $$38 := %0->$$10]) decor ([]) {
+                      aggregate [$$29, $$30, $$31, $$32, $$33, $$34, $$35, $$36] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$5], function-call: hive:sum(PARTIAL1), Args:[%0->$$6], function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$6, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$7]]], function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$6, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$7]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPPlus, Args:[1, %0->$$8]]], function-call: hive:avg(PARTIAL1), Args:[%0->$$5], function-call: hive:avg(PARTIAL1), Args:[%0->$$6], function-call: hive:avg(PARTIAL1), Args:[%0->$$7], function-call: hive:count(PARTIAL1), Args:[1]]
                       -- AGGREGATE  |LOCAL|
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$37, $$38]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$9, $$10]  |PARTITIONED|
               exchange 
-              -- HASH_PARTITION_EXCHANGE [$$37, $$38]  |PARTITIONED|
-                group by ([$$37 := %0->$$9; $$38 := %0->$$10]) decor ([]) {
-                          aggregate [$$29, $$30, $$31, $$32, $$33, $$34, $$35, $$36] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$5], function-call: hive:sum(PARTIAL1), Args:[%0->$$6], function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$6, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$7]]], function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$6, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$7]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPPlus, Args:[1, %0->$$8]]], function-call: hive:avg(PARTIAL1), Args:[%0->$$5], function-call: hive:avg(PARTIAL1), Args:[%0->$$6], function-call: hive:avg(PARTIAL1), Args:[%0->$$7], function-call: hive:count(PARTIAL1), Args:[1]]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       }
-                -- EXTERNAL_GROUP_BY[$$9, $$10]  |PARTITIONED|
-                  exchange 
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$9, $$10, $$5, $$6, $$7, $$8])
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (function-call: algebricks:le, Args:[%0->$$11, 1998-09-02])
-                      -- STREAM_SELECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$9, $$10, $$5, $$6, $$7, $$8])
+                -- STREAM_PROJECT  |PARTITIONED|
+                  select (function-call: algebricks:le, Args:[%0->$$11, 1998-09-02])
+                  -- STREAM_SELECT  |PARTITIONED|
+                    exchange 
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan [$$5, $$6, $$7, $$8, $$9, $$10, $$11]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
+                      -- DATASOURCE_SCAN  |PARTITIONED|
                         exchange 
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan [$$5, $$6, $$7, $$8, $$9, $$10, $$11]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
-                          -- DATASOURCE_SCAN  |PARTITIONED|
-                            exchange 
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              empty-tuple-source
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                          empty-tuple-source
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan
index ded599c..cecf482 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q20_potential_part_promotion.plan
@@ -36,16 +36,16 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[$$23, $$24]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[$$23, $$24]  |PARTITIONED|
           exchange 
-          -- HASH_PARTITION_EXCHANGE [$$23, $$24]  |PARTITIONED|
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$23(ASC), $$24(ASC)] HASH:[$$23, $$24]  |PARTITIONED|
             group by ([$$23 := %0->$$2; $$24 := %0->$$3]) decor ([]) {
                       aggregate [$$22] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$5]]
                       -- AGGREGATE  |LOCAL|
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$2, $$3]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$2, $$3]  |PARTITIONED|
               exchange 
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 project ([$$2, $$3, $$5])
@@ -175,4 +175,4 @@
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             empty-tuple-source
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
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 e4f2cd6..28bc66b 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
@@ -21,19 +21,15 @@
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- PRE_CLUSTERED_GROUP_BY[$$1, $$3]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$1, $$3]  |PARTITIONED|
               exchange 
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, %0->$$1) (ASC, %0->$$3) 
-                -- STABLE_SORT [$$1(ASC), $$3(ASC)]  |PARTITIONED|
+                data-scan [$$1, $$3]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
+                -- DATASOURCE_SCAN  |PARTITIONED|
                   exchange 
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan [$$1, $$3]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
-                    -- DATASOURCE_SCAN  |PARTITIONED|
-                      exchange 
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        empty-tuple-source
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    empty-tuple-source
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 write [%0->$$21, %0->$$24, %0->$$23]
 -- SINK_WRITE  |PARTITIONED|
   project ([$$21, $$24, $$23])
@@ -57,25 +53,21 @@
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- PRE_CLUSTERED_GROUP_BY[$$1, $$3]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$1, $$3]  |PARTITIONED|
               exchange 
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                order (ASC, %0->$$1) (ASC, %0->$$3) 
-                -- STABLE_SORT [$$1(ASC), $$3(ASC)]  |PARTITIONED|
-                  exchange 
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$1, $$3])
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      select (function-call: algebricks:gt, Args:[%0->$$13, %0->$$12])
-                      -- STREAM_SELECT  |PARTITIONED|
+                project ([$$1, $$3])
+                -- STREAM_PROJECT  |PARTITIONED|
+                  select (function-call: algebricks:gt, Args:[%0->$$13, %0->$$12])
+                  -- STREAM_SELECT  |PARTITIONED|
+                    exchange 
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan [$$1, $$3, $$12, $$13]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
+                      -- DATASOURCE_SCAN  |PARTITIONED|
                         exchange 
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan [$$1, $$3, $$12, $$13]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10, $$11, $$12, $$13, $$14, $$15, $$16] <- default.lineitem
-                          -- DATASOURCE_SCAN  |PARTITIONED|
-                            exchange 
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              empty-tuple-source
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          empty-tuple-source
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 write [%0->$$43, %0->$$45]
 -- SINK_WRITE  |UNPARTITIONED|
   project ([$$43, $$45])
@@ -102,16 +94,16 @@
                                 nested tuple source
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                            }
-                    -- EXTERNAL_GROUP_BY[$$48]  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$48]  |PARTITIONED|
                       exchange 
-                      -- HASH_PARTITION_EXCHANGE [$$48]  |PARTITIONED|
+                      -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$48(ASC)] HASH:[$$48]  |PARTITIONED|
                         group by ([$$48 := %0->$$37]) decor ([]) {
                                   aggregate [$$47] <- [function-call: hive:count(PARTIAL1), Args:[1]]
                                   -- AGGREGATE  |LOCAL|
                                     nested tuple source
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                }
-                        -- EXTERNAL_GROUP_BY[$$37]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$37]  |PARTITIONED|
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             project ([$$37])
@@ -221,4 +213,4 @@
                                         exchange 
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           empty-tuple-source
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q22_global_sales_opportunity.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q22_global_sales_opportunity.plan
index c5897f7..15208bd 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q22_global_sales_opportunity.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q22_global_sales_opportunity.plan
@@ -24,7 +24,7 @@
                 nested tuple source
                 -- NESTED_TUPLE_SOURCE  |LOCAL|
            }
-    -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+    -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
       exchange 
       -- HASH_PARTITION_EXCHANGE []  |PARTITIONED|
         group by ([]) decor ([]) {
@@ -33,7 +33,7 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
           exchange 
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             select (function-call: algebricks:gt, Args:[%0->$$1, 0.0])
@@ -72,65 +72,61 @@
     -- ASSIGN  |PARTITIONED|
       exchange 
       -- SORT_MERGE_EXCHANGE [$$6(ASC) ]  |PARTITIONED|
-        order (ASC, %0->$$6) 
-        -- STABLE_SORT [$$6(ASC)]  |PARTITIONED|
+        group by ([$$6 := %0->$$13]) decor ([]) {
+                  aggregate [$$7, $$8] <- [function-call: hive:count(FINAL), Args:[%0->$$11], function-call: hive:sum(FINAL), Args:[%0->$$12]]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               }
+        -- PRE_CLUSTERED_GROUP_BY[$$13]  |PARTITIONED|
           exchange 
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$6 := %0->$$13]) decor ([]) {
-                      aggregate [$$7, $$8] <- [function-call: hive:count(FINAL), Args:[%0->$$11], function-call: hive:sum(FINAL), Args:[%0->$$12]]
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$13(ASC)] HASH:[$$13]  |PARTITIONED|
+            group by ([$$13 := %0->$$5]) decor ([]) {
+                      aggregate [$$11, $$12] <- [function-call: hive:count(PARTIAL1), Args:[1], function-call: hive:sum(PARTIAL1), Args:[%0->$$3]]
                       -- AGGREGATE  |LOCAL|
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$13]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$5]  |PARTITIONED|
               exchange 
-              -- HASH_PARTITION_EXCHANGE [$$13]  |PARTITIONED|
-                group by ([$$13 := %0->$$5]) decor ([]) {
-                          aggregate [$$11, $$12] <- [function-call: hive:count(PARTIAL1), Args:[1], function-call: hive:sum(PARTIAL1), Args:[%0->$$3]]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       }
-                -- EXTERNAL_GROUP_BY[$$5]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$5, $$3])
+                -- STREAM_PROJECT  |PARTITIONED|
                   exchange 
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$5, $$3])
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    join (function-call: algebricks:and, Args:[function-call: algebricks:gt, Args:[%0->$$3, %0->$$1], true])
+                    -- NESTED_LOOP  |PARTITIONED|
+                      exchange 
+                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        project ([$$5, $$3])
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          select (function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull, Args:[%0->$$2])
+                          -- STREAM_SELECT  |PARTITIONED|
+                            exchange 
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              left outer join (function-call: algebricks:eq, Args:[%0->$$4, %0->$$2])
+                              -- HYBRID_HASH_JOIN [$$4][$$2]  |PARTITIONED|
+                                exchange 
+                                -- HASH_PARTITION_EXCHANGE [$$4]  |PARTITIONED|
+                                  data-scan []<-[$$3, $$4, $$5] <- default.q22_customer_tmp
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    exchange 
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      empty-tuple-source
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                exchange 
+                                -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                                  data-scan [$$2]<-[$$2] <- default.q22_orders_tmp
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    exchange 
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      empty-tuple-source
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                       exchange 
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        join (function-call: algebricks:and, Args:[function-call: algebricks:gt, Args:[%0->$$3, %0->$$1], true])
-                        -- NESTED_LOOP  |PARTITIONED|
-                          exchange 
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            project ([$$5, $$3])
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              select (function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull, Args:[%0->$$2])
-                              -- STREAM_SELECT  |PARTITIONED|
-                                exchange 
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  left outer join (function-call: algebricks:eq, Args:[%0->$$4, %0->$$2])
-                                  -- HYBRID_HASH_JOIN [$$4][$$2]  |PARTITIONED|
-                                    exchange 
-                                    -- HASH_PARTITION_EXCHANGE [$$4]  |PARTITIONED|
-                                      data-scan []<-[$$3, $$4, $$5] <- default.q22_customer_tmp
-                                      -- DATASOURCE_SCAN  |PARTITIONED|
-                                        exchange 
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          empty-tuple-source
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    exchange 
-                                    -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
-                                      data-scan [$$2]<-[$$2] <- default.q22_orders_tmp
-                                      -- DATASOURCE_SCAN  |PARTITIONED|
-                                        exchange 
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          empty-tuple-source
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        data-scan [$$1]<-[$$1] <- default.q22_customer_tmp1
+                        -- DATASOURCE_SCAN  |PARTITIONED|
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            data-scan [$$1]<-[$$1] <- default.q22_customer_tmp1
-                            -- DATASOURCE_SCAN  |PARTITIONED|
-                              exchange 
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            empty-tuple-source
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan
index 6138f7a..2714806 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q2_minimum_cost_supplier.plan
@@ -103,19 +103,15 @@
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- PRE_CLUSTERED_GROUP_BY[$$4]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$4]  |PARTITIONED|
             exchange 
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              order (ASC, %0->$$4) 
-              -- STABLE_SORT [$$4(ASC)]  |PARTITIONED|
+              data-scan [$$4, $$5]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.q2_minimum_cost_supplier_tmp1
+              -- DATASOURCE_SCAN  |PARTITIONED|
                 exchange 
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  data-scan [$$4, $$5]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.q2_minimum_cost_supplier_tmp1
-                  -- DATASOURCE_SCAN  |PARTITIONED|
-                    exchange 
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      empty-tuple-source
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
 write [%0->$$3, %0->$$4, %0->$$5, %0->$$6, %0->$$8, %0->$$9, %0->$$10, %0->$$11]
 -- SINK_WRITE  |UNPARTITIONED|
   limit 100
@@ -153,4 +149,4 @@
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             empty-tuple-source
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q3_shipping_priority.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q3_shipping_priority.plan
index 31c4210..05563e2 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q3_shipping_priority.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q3_shipping_priority.plan
@@ -67,4 +67,4 @@
                                             exchange 
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               empty-tuple-source
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan
index be5a66a..23a64fe 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q4_order_priority.plan
@@ -30,53 +30,49 @@
     -- ASSIGN  |PARTITIONED|
       exchange 
       -- SORT_MERGE_EXCHANGE [$$11(ASC) ]  |PARTITIONED|
-        order (ASC, %0->$$11) 
-        -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
+        group by ([$$11 := %0->$$16]) decor ([]) {
+                  aggregate [$$12] <- [function-call: hive:count(FINAL), Args:[%0->$$15]]
+                  -- AGGREGATE  |LOCAL|
+                    nested tuple source
+                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+               }
+        -- PRE_CLUSTERED_GROUP_BY[$$16]  |PARTITIONED|
           exchange 
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            group by ([$$11 := %0->$$16]) decor ([]) {
-                      aggregate [$$12] <- [function-call: hive:count(FINAL), Args:[%0->$$15]]
+          -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$16(ASC)] HASH:[$$16]  |PARTITIONED|
+            group by ([$$16 := %0->$$7]) decor ([]) {
+                      aggregate [$$15] <- [function-call: hive:count(PARTIAL1), Args:[1]]
                       -- AGGREGATE  |LOCAL|
                         nested tuple source
                         -- NESTED_TUPLE_SOURCE  |LOCAL|
                    }
-            -- EXTERNAL_GROUP_BY[$$16]  |PARTITIONED|
+            -- SORT_GROUP_BY[$$7]  |PARTITIONED|
               exchange 
-              -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
-                group by ([$$16 := %0->$$7]) decor ([]) {
-                          aggregate [$$15] <- [function-call: hive:count(PARTIAL1), Args:[1]]
-                          -- AGGREGATE  |LOCAL|
-                            nested tuple source
-                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                       }
-                -- EXTERNAL_GROUP_BY[$$7]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$7])
+                -- STREAM_PROJECT  |PARTITIONED|
                   exchange 
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    project ([$$7])
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    join (function-call: algebricks:eq, Args:[%0->$$1, %0->$$2])
+                    -- HYBRID_HASH_JOIN [$$1][$$2]  |PARTITIONED|
                       exchange 
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        join (function-call: algebricks:eq, Args:[%0->$$1, %0->$$2])
-                        -- HYBRID_HASH_JOIN [$$1][$$2]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
+                        data-scan []<-[$$1] <- default.q4_order_priority_tmp
+                        -- DATASOURCE_SCAN  |PARTITIONED|
                           exchange 
-                          -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
-                            data-scan []<-[$$1] <- default.q4_order_priority_tmp
-                            -- DATASOURCE_SCAN  |PARTITIONED|
-                              exchange 
-                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                empty-tuple-source
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                          exchange 
-                          -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
-                            project ([$$2, $$7])
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$6, 1993-07-01], function-call: algebricks:lt, Args:[%0->$$6, 1993-10-01]])
-                              -- STREAM_SELECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            empty-tuple-source
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      exchange 
+                      -- HASH_PARTITION_EXCHANGE [$$2]  |PARTITIONED|
+                        project ([$$2, $$7])
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$6, 1993-07-01], function-call: algebricks:lt, Args:[%0->$$6, 1993-10-01]])
+                          -- STREAM_SELECT  |PARTITIONED|
+                            exchange 
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan [$$2, $$6, $$7]<-[$$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10] <- default.orders
+                              -- DATASOURCE_SCAN  |PARTITIONED|
                                 exchange 
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  data-scan [$$2, $$6, $$7]<-[$$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9, $$10] <- default.orders
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
-                                    exchange 
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      empty-tuple-source
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                  empty-tuple-source
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan
index 383e550..a0f5b8c 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q5_local_supplier_volume.plan
@@ -14,16 +14,16 @@
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- EXTERNAL_GROUP_BY[$$52]  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$52]  |PARTITIONED|
             exchange 
-            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+            -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$52(ASC)] HASH:[$$52]  |PARTITIONED|
               group by ([$$52 := %0->$$42]) decor ([]) {
                         aggregate [$$51] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$23, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$24]]]]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- EXTERNAL_GROUP_BY[$$42]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$42]  |PARTITIONED|
                 exchange 
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   project ([$$42, $$23, $$24])
@@ -123,4 +123,4 @@
                             exchange 
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               empty-tuple-source
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan
index aac9a5b..03363f3 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q6_forecast_revenue_change.plan
@@ -8,7 +8,7 @@
                 nested tuple source
                 -- NESTED_TUPLE_SOURCE  |LOCAL|
            }
-    -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+    -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
       exchange 
       -- HASH_PARTITION_EXCHANGE []  |PARTITIONED|
         group by ([]) decor ([]) {
@@ -17,7 +17,7 @@
                     nested tuple source
                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                }
-        -- EXTERNAL_GROUP_BY[]  |PARTITIONED|
+        -- PRE_CLUSTERED_GROUP_BY[]  |PARTITIONED|
           exchange 
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             project ([$$6, $$7])
@@ -31,4 +31,4 @@
                     exchange 
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       empty-tuple-source
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan
index c1d5b26..9f62e7b 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q7_volume_shipping.plan
@@ -90,103 +90,99 @@
   -- STREAM_PROJECT  |PARTITIONED|
     exchange 
     -- SORT_MERGE_EXCHANGE [$$47(ASC), $$48(ASC), $$49(ASC) ]  |PARTITIONED|
-      order (ASC, %0->$$47) (ASC, %0->$$48) (ASC, %0->$$49) 
-      -- STABLE_SORT [$$47(ASC), $$48(ASC), $$49(ASC)]  |PARTITIONED|
+      group by ([$$47 := %0->$$53; $$48 := %0->$$54; $$49 := %0->$$55]) decor ([]) {
+                aggregate [$$50] <- [function-call: hive:sum(FINAL), Args:[%0->$$52]]
+                -- AGGREGATE  |LOCAL|
+                  nested tuple source
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+             }
+      -- PRE_CLUSTERED_GROUP_BY[$$53, $$54, $$55]  |PARTITIONED|
         exchange 
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$47 := %0->$$53; $$48 := %0->$$54; $$49 := %0->$$55]) decor ([]) {
-                    aggregate [$$50] <- [function-call: hive:sum(FINAL), Args:[%0->$$52]]
+        -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$53(ASC), $$54(ASC), $$55(ASC)] HASH:[$$53, $$54, $$55]  |PARTITIONED|
+          group by ([$$53 := %0->$$1; $$54 := %0->$$2; $$55 := %0->$$45]) decor ([]) {
+                    aggregate [$$52] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$46]]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- EXTERNAL_GROUP_BY[$$53, $$54, $$55]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$1, $$2, $$45]  |PARTITIONED|
             exchange 
-            -- HASH_PARTITION_EXCHANGE [$$53, $$54, $$55]  |PARTITIONED|
-              group by ([$$53 := %0->$$1; $$54 := %0->$$2; $$55 := %0->$$45]) decor ([]) {
-                        aggregate [$$52] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$46]]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     }
-              -- EXTERNAL_GROUP_BY[$$1, $$2, $$45]  |PARTITIONED|
-                exchange 
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$1, $$2, $$45, $$46])
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$1, $$2, $$45, $$46])
+              -- STREAM_PROJECT  |PARTITIONED|
+                assign [$$45, $$46] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFYear, Args:[%0->$$30], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$25, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$26]]]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$1, $$2, $$30, $$25, $$26])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$45, $$46] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFYear, Args:[%0->$$30], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$25, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$26]]]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$1, $$2, $$30, $$25, $$26])
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    exchange 
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      join (function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$15, %0->$$4], function-call: algebricks:eq, Args:[%0->$$8, %0->$$3]])
+                      -- HYBRID_HASH_JOIN [$$15, $$8][$$4, $$3]  |PARTITIONED|
                         exchange 
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          join (function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$15, %0->$$4], function-call: algebricks:eq, Args:[%0->$$8, %0->$$3]])
-                          -- HYBRID_HASH_JOIN [$$15, $$8][$$4, $$3]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$15, $$8]  |PARTITIONED|
+                          project ([$$8, $$30, $$25, $$26, $$15])
+                          -- STREAM_PROJECT  |PARTITIONED|
                             exchange 
-                            -- HASH_PARTITION_EXCHANGE [$$15, $$8]  |PARTITIONED|
-                              project ([$$8, $$30, $$25, $$26, $$15])
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              join (function-call: algebricks:eq, Args:[%0->$$22, %0->$$5])
+                              -- HYBRID_HASH_JOIN [$$22][$$5]  |PARTITIONED|
                                 exchange 
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  join (function-call: algebricks:eq, Args:[%0->$$22, %0->$$5])
-                                  -- HYBRID_HASH_JOIN [$$22][$$5]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                  project ([$$15, $$30, $$25, $$26, $$22])
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     exchange 
-                                    -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                      project ([$$15, $$30, $$25, $$26, $$22])
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      join (function-call: algebricks:eq, Args:[%0->$$37, %0->$$12])
+                                      -- HYBRID_HASH_JOIN [$$37][$$12]  |PARTITIONED|
                                         exchange 
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          join (function-call: algebricks:eq, Args:[%0->$$37, %0->$$12])
-                                          -- HYBRID_HASH_JOIN [$$37][$$12]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                          project ([$$37, $$22, $$25, $$26, $$30])
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             exchange 
-                                            -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                                              project ([$$37, $$22, $$25, $$26, $$30])
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              join (function-call: algebricks:eq, Args:[%0->$$20, %0->$$36])
+                                              -- HYBRID_HASH_JOIN [$$20][$$36]  |PARTITIONED|
                                                 exchange 
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  join (function-call: algebricks:eq, Args:[%0->$$20, %0->$$36])
-                                                  -- HYBRID_HASH_JOIN [$$20][$$36]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                                  select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$30, 1995-01-01], function-call: algebricks:le, Args:[%0->$$30, 1996-12-31]])
+                                                  -- STREAM_SELECT  |PARTITIONED|
                                                     exchange 
-                                                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                                                      select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$30, 1995-01-01], function-call: algebricks:le, Args:[%0->$$30, 1996-12-31]])
-                                                      -- STREAM_SELECT  |PARTITIONED|
-                                                        exchange 
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          data-scan [$$20, $$22, $$25, $$26, $$30]<-[$$20, $$21, $$22, $$23, $$24, $$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33, $$34, $$35] <- default.lineitem
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            exchange 
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              empty-tuple-source
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    exchange 
-                                                    -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
-                                                      data-scan [$$36, $$37]<-[$$36, $$37, $$38, $$39, $$40, $$41, $$42, $$43, $$44] <- default.orders
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      data-scan [$$20, $$22, $$25, $$26, $$30]<-[$$20, $$21, $$22, $$23, $$24, $$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33, $$34, $$35] <- default.lineitem
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                                         exchange 
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           empty-tuple-source
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                            exchange 
-                                            -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
-                                              data-scan [$$12, $$15]<-[$$12, $$13, $$14, $$15, $$16, $$17, $$18, $$19] <- default.customer
-                                              -- DATASOURCE_SCAN  |PARTITIONED|
                                                 exchange 
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  empty-tuple-source
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    exchange 
-                                    -- HASH_PARTITION_EXCHANGE [$$5]  |PARTITIONED|
-                                      data-scan [$$5, $$8]<-[$$5, $$6, $$7, $$8, $$9, $$10, $$11] <- default.supplier
-                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$36]  |PARTITIONED|
+                                                  data-scan [$$36, $$37]<-[$$36, $$37, $$38, $$39, $$40, $$41, $$42, $$43, $$44] <- default.orders
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    exchange 
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      empty-tuple-source
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                         exchange 
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          empty-tuple-source
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            exchange 
-                            -- HASH_PARTITION_EXCHANGE [$$4, $$3]  |PARTITIONED|
-                              data-scan [$$4, $$3, $$1, $$2]<-[$$1, $$2, $$3, $$4] <- default.q7_volume_shipping_tmp
-                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+                                          data-scan [$$12, $$15]<-[$$12, $$13, $$14, $$15, $$16, $$17, $$18, $$19] <- default.customer
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            exchange 
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              empty-tuple-source
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                 exchange 
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  empty-tuple-source
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                -- HASH_PARTITION_EXCHANGE [$$5]  |PARTITIONED|
+                                  data-scan [$$5, $$8]<-[$$5, $$6, $$7, $$8, $$9, $$10, $$11] <- default.supplier
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    exchange 
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      empty-tuple-source
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        exchange 
+                        -- HASH_PARTITION_EXCHANGE [$$4, $$3]  |PARTITIONED|
+                          data-scan [$$4, $$3, $$1, $$2]<-[$$1, $$2, $$3, $$4] <- default.q7_volume_shipping_tmp
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange 
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              empty-tuple-source
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan
index b9916e2..e02580a 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q8_national_market_share.plan
@@ -2,189 +2,185 @@
 -- SINK_WRITE  |PARTITIONED|
   exchange 
   -- SORT_MERGE_EXCHANGE [$$63(ASC) ]  |PARTITIONED|
-    order (ASC, %0->$$63) 
-    -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
-      exchange 
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        project ([$$63, $$66])
-        -- STREAM_PROJECT  |PARTITIONED|
-          assign [$$66] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPDivide, Args:[%0->$$64, %0->$$65]]
-          -- ASSIGN  |PARTITIONED|
+    project ([$$63, $$66])
+    -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$66] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPDivide, Args:[%0->$$64, %0->$$65]]
+      -- ASSIGN  |PARTITIONED|
+        exchange 
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          group by ([$$63 := %0->$$70]) decor ([]) {
+                    aggregate [$$64, $$65] <- [function-call: hive:sum(FINAL), Args:[%0->$$68], function-call: hive:sum(FINAL), Args:[%0->$$69]]
+                    -- AGGREGATE  |LOCAL|
+                      nested tuple source
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                 }
+          -- PRE_CLUSTERED_GROUP_BY[$$70]  |PARTITIONED|
             exchange 
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              group by ([$$63 := %0->$$70]) decor ([]) {
-                        aggregate [$$64, $$65] <- [function-call: hive:sum(FINAL), Args:[%0->$$68], function-call: hive:sum(FINAL), Args:[%0->$$69]]
+            -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$70(ASC)] HASH:[$$70]  |PARTITIONED|
+              group by ([$$70 := %0->$$61]) decor ([]) {
+                        aggregate [$$68, $$69] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen, Args:[function-call: algebricks:eq, Args:[%0->$$2, BRAZIL], %0->$$62, 0.0]], function-call: hive:sum(PARTIAL1), Args:[%0->$$62]]
                         -- AGGREGATE  |LOCAL|
                           nested tuple source
                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                      }
-              -- EXTERNAL_GROUP_BY[$$70]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$61]  |PARTITIONED|
                 exchange 
-                -- HASH_PARTITION_EXCHANGE [$$70]  |PARTITIONED|
-                  group by ([$$70 := %0->$$61]) decor ([]) {
-                            aggregate [$$68, $$69] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen, Args:[function-call: algebricks:eq, Args:[%0->$$2, BRAZIL], %0->$$62, 0.0]], function-call: hive:sum(PARTIAL1), Args:[%0->$$62]]
-                            -- AGGREGATE  |LOCAL|
-                              nested tuple source
-                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                         }
-                  -- EXTERNAL_GROUP_BY[$$61]  |PARTITIONED|
-                    exchange 
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      project ([$$61, $$62, $$2])
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  project ([$$61, $$62, $$2])
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    assign [$$61, $$62] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFYear, Args:[%0->$$41], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$26, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$27]]]
+                    -- ASSIGN  |PARTITIONED|
+                      project ([$$2, $$41, $$27, $$26])
                       -- STREAM_PROJECT  |PARTITIONED|
-                        assign [$$61, $$62] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFYear, Args:[%0->$$41], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$26, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$27]]]
-                        -- ASSIGN  |PARTITIONED|
-                          project ([$$2, $$41, $$27, $$26])
-                          -- STREAM_PROJECT  |PARTITIONED|
+                        exchange 
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          join (function-call: algebricks:eq, Args:[%0->$$8, %0->$$1])
+                          -- HYBRID_HASH_JOIN [$$8][$$1]  |PARTITIONED|
                             exchange 
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              join (function-call: algebricks:eq, Args:[%0->$$8, %0->$$1])
-                              -- HYBRID_HASH_JOIN [$$8][$$1]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$8]  |PARTITIONED|
+                              project ([$$8, $$41, $$27, $$26])
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 exchange 
-                                -- HASH_PARTITION_EXCHANGE [$$8]  |PARTITIONED|
-                                  project ([$$8, $$41, $$27, $$26])
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  join (function-call: algebricks:eq, Args:[%0->$$23, %0->$$5])
+                                  -- HYBRID_HASH_JOIN [$$23][$$5]  |PARTITIONED|
                                     exchange 
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      join (function-call: algebricks:eq, Args:[%0->$$23, %0->$$5])
-                                      -- HYBRID_HASH_JOIN [$$23][$$5]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+                                      project ([$$41, $$27, $$26, $$23])
+                                      -- STREAM_PROJECT  |PARTITIONED|
                                         exchange 
-                                        -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
-                                          project ([$$41, $$27, $$26, $$23])
-                                          -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          join (function-call: algebricks:eq, Args:[%0->$$22, %0->$$12])
+                                          -- HYBRID_HASH_JOIN [$$22][$$12]  |PARTITIONED|
                                             exchange 
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              join (function-call: algebricks:eq, Args:[%0->$$22, %0->$$12])
-                                              -- HYBRID_HASH_JOIN [$$22][$$12]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
+                                              project ([$$41, $$22, $$23, $$26, $$27])
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 exchange 
-                                                -- HASH_PARTITION_EXCHANGE [$$22]  |PARTITIONED|
-                                                  project ([$$41, $$22, $$23, $$26, $$27])
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  join (function-call: algebricks:eq, Args:[%0->$$21, %0->$$37])
+                                                  -- HYBRID_HASH_JOIN [$$21][$$37]  |PARTITIONED|
                                                     exchange 
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      join (function-call: algebricks:eq, Args:[%0->$$21, %0->$$37])
-                                                      -- HYBRID_HASH_JOIN [$$21][$$37]  |PARTITIONED|
+                                                    -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
+                                                      data-scan [$$21, $$22, $$23, $$26, $$27]<-[$$21, $$22, $$23, $$24, $$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33, $$34, $$35, $$36] <- default.lineitem
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                                         exchange 
-                                                        -- HASH_PARTITION_EXCHANGE [$$21]  |PARTITIONED|
-                                                          data-scan [$$21, $$22, $$23, $$26, $$27]<-[$$21, $$22, $$23, $$24, $$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33, $$34, $$35, $$36] <- default.lineitem
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            exchange 
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              empty-tuple-source
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          empty-tuple-source
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    exchange 
+                                                    -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
+                                                      project ([$$37, $$41])
+                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                         exchange 
-                                                        -- HASH_PARTITION_EXCHANGE [$$37]  |PARTITIONED|
-                                                          project ([$$37, $$41])
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          join (function-call: algebricks:eq, Args:[%0->$$46, %0->$$38])
+                                                          -- HYBRID_HASH_JOIN [$$46][$$38]  |PARTITIONED|
                                                             exchange 
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              join (function-call: algebricks:eq, Args:[%0->$$46, %0->$$38])
-                                                              -- HYBRID_HASH_JOIN [$$46][$$38]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
+                                                              project ([$$46])
+                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                 exchange 
-                                                                -- HASH_PARTITION_EXCHANGE [$$46]  |PARTITIONED|
-                                                                  project ([$$46])
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  join (function-call: algebricks:eq, Args:[%0->$$54, %0->$$49])
+                                                                  -- HYBRID_HASH_JOIN [$$54][$$49]  |PARTITIONED|
                                                                     exchange 
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      join (function-call: algebricks:eq, Args:[%0->$$54, %0->$$49])
-                                                                      -- HYBRID_HASH_JOIN [$$54][$$49]  |PARTITIONED|
+                                                                    -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                                                      project ([$$54])
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
                                                                         exchange 
-                                                                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
-                                                                          project ([$$54])
-                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          join (function-call: algebricks:eq, Args:[%0->$$58, %0->$$56])
+                                                                          -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
                                                                             exchange 
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              join (function-call: algebricks:eq, Args:[%0->$$58, %0->$$56])
-                                                                              -- HYBRID_HASH_JOIN [$$58][$$56]  |PARTITIONED|
-                                                                                exchange 
-                                                                                -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
-                                                                                  project ([$$58])
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                    select (function-call: algebricks:eq, Args:[%0->$$59, AMERICA])
-                                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                            -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                                                                              project ([$$58])
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                select (function-call: algebricks:eq, Args:[%0->$$59, AMERICA])
+                                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                                  exchange 
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    data-scan [$$59, $$58]<-[$$58, $$59, $$60] <- default.region
+                                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                       exchange 
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                        data-scan [$$59, $$58]<-[$$58, $$59, $$60] <- default.region
-                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                                          exchange 
-                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                            empty-tuple-source
-                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                        empty-tuple-source
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                            exchange 
+                                                                            -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                                                                              project ([$$56, $$54])
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
                                                                                 exchange 
-                                                                                -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
-                                                                                  project ([$$56, $$54])
-                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  replicate 
+                                                                                  -- SPLIT  |PARTITIONED|
                                                                                     exchange 
                                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      replicate 
-                                                                                      -- SPLIT  |PARTITIONED|
+                                                                                      data-scan []<-[$$54, $$55, $$56, $$57] <- default.nation
+                                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                                                                         exchange 
                                                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          data-scan []<-[$$54, $$55, $$56, $$57] <- default.nation
-                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                                            exchange 
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              empty-tuple-source
-                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          empty-tuple-source
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                    exchange 
+                                                                    -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
+                                                                      data-scan [$$49, $$46]<-[$$46, $$47, $$48, $$49, $$50, $$51, $$52, $$53] <- default.customer
+                                                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                                                         exchange 
-                                                                        -- HASH_PARTITION_EXCHANGE [$$49]  |PARTITIONED|
-                                                                          data-scan [$$49, $$46]<-[$$46, $$47, $$48, $$49, $$50, $$51, $$52, $$53] <- default.customer
-                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                            exchange 
-                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                              empty-tuple-source
-                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                exchange 
-                                                                -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
-                                                                  project ([$$38, $$37, $$41])
-                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                    select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$41, 1995-01-01], function-call: algebricks:lt, Args:[%0->$$41, 1996-12-31]])
-                                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          empty-tuple-source
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            exchange 
+                                                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                                              project ([$$38, $$37, $$41])
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                select (function-call: algebricks:and, Args:[function-call: algebricks:ge, Args:[%0->$$41, 1995-01-01], function-call: algebricks:lt, Args:[%0->$$41, 1996-12-31]])
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  exchange 
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    data-scan [$$38, $$37, $$41]<-[$$37, $$38, $$39, $$40, $$41, $$42, $$43, $$44, $$45] <- default.orders
+                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                                                       exchange 
                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                        data-scan [$$38, $$37, $$41]<-[$$37, $$38, $$39, $$40, $$41, $$42, $$43, $$44, $$45] <- default.orders
-                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                          exchange 
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            empty-tuple-source
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                exchange 
-                                                -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
-                                                  project ([$$12])
-                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                    select (function-call: algebricks:eq, Args:[%0->$$16, ECONOMY ANODIZED STEEL])
-                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                                        empty-tuple-source
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            exchange 
+                                            -- HASH_PARTITION_EXCHANGE [$$12]  |PARTITIONED|
+                                              project ([$$12])
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                select (function-call: algebricks:eq, Args:[%0->$$16, ECONOMY ANODIZED STEEL])
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  exchange 
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    data-scan [$$16, $$12]<-[$$12, $$13, $$14, $$15, $$16, $$17, $$18, $$19, $$20] <- default.part
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                                       exchange 
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        data-scan [$$16, $$12]<-[$$12, $$13, $$14, $$15, $$16, $$17, $$18, $$19, $$20] <- default.part
-                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                          exchange 
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            empty-tuple-source
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        empty-tuple-source
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    exchange 
+                                    -- HASH_PARTITION_EXCHANGE [$$5]  |PARTITIONED|
+                                      data-scan [$$5, $$8]<-[$$5, $$6, $$7, $$8, $$9, $$10, $$11] <- default.supplier
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         exchange 
-                                        -- HASH_PARTITION_EXCHANGE [$$5]  |PARTITIONED|
-                                          data-scan [$$5, $$8]<-[$$5, $$6, $$7, $$8, $$9, $$10, $$11] <- default.supplier
-                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                            exchange 
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              empty-tuple-source
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                exchange 
-                                -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
-                                  project ([$$1, $$2])
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    assign [$$1, $$2, $$3, $$4] <- [%0->$$54, %0->$$55, %0->$$56, %0->$$57]
-                                    -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          empty-tuple-source
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            exchange 
+                            -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
+                              project ([$$1, $$2])
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                assign [$$1, $$2, $$3, $$4] <- [%0->$$54, %0->$$55, %0->$$56, %0->$$57]
+                                -- ASSIGN  |PARTITIONED|
+                                  exchange 
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    replicate 
+                                    -- SPLIT  |PARTITIONED|
                                       exchange 
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        replicate 
-                                        -- SPLIT  |PARTITIONED|
+                                        data-scan []<-[$$54, $$55, $$56, $$57] <- default.nation
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
                                           exchange 
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan []<-[$$54, $$55, $$56, $$57] <- default.nation
-                                            -- DATASOURCE_SCAN  |PARTITIONED|
-                                              exchange 
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                empty-tuple-source
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                            empty-tuple-source
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q9_product_type_profit.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q9_product_type_profit.plan
index ecf4acb..c5d8052 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q9_product_type_profit.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q9_product_type_profit.plan
@@ -4,121 +4,117 @@
   -- STREAM_PROJECT  |PARTITIONED|
     exchange 
     -- SORT_MERGE_EXCHANGE [$$53(ASC), $$54(DESC) ]  |PARTITIONED|
-      order (ASC, %0->$$53) (DESC, %0->$$54) 
-      -- STABLE_SORT [$$53(ASC), $$54(DESC)]  |PARTITIONED|
+      group by ([$$53 := %0->$$58; $$54 := %0->$$59]) decor ([]) {
+                aggregate [$$55] <- [function-call: hive:sum(FINAL), Args:[%0->$$57]]
+                -- AGGREGATE  |LOCAL|
+                  nested tuple source
+                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+             }
+      -- PRE_CLUSTERED_GROUP_BY[$$58, $$59]  |PARTITIONED|
         exchange 
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          group by ([$$53 := %0->$$58; $$54 := %0->$$59]) decor ([]) {
-                    aggregate [$$55] <- [function-call: hive:sum(FINAL), Args:[%0->$$57]]
+        -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$58(ASC), $$59(DESC)] HASH:[$$58, $$59]  |PARTITIONED|
+          group by ([$$58 := %0->$$48; $$59 := %0->$$51]) decor ([]) {
+                    aggregate [$$57] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$52]]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- EXTERNAL_GROUP_BY[$$58, $$59]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$48, $$51]  |PARTITIONED|
             exchange 
-            -- HASH_PARTITION_EXCHANGE [$$58, $$59]  |PARTITIONED|
-              group by ([$$58 := %0->$$48; $$59 := %0->$$51]) decor ([]) {
-                        aggregate [$$57] <- [function-call: hive:sum(PARTIAL1), Args:[%0->$$52]]
-                        -- AGGREGATE  |LOCAL|
-                          nested tuple source
-                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                     }
-              -- EXTERNAL_GROUP_BY[$$48, $$51]  |PARTITIONED|
-                exchange 
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  project ([$$48, $$51, $$52])
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$48, $$51, $$52])
+              -- STREAM_PROJECT  |PARTITIONED|
+                assign [$$51, $$52] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFYear, Args:[%0->$$5], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$29, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$30]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$22, %0->$$28]]]
+                -- ASSIGN  |PARTITIONED|
+                  project ([$$5, $$29, $$30, $$28, $$48, $$22])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$51, $$52] <- [function-call: hive:org.apache.hadoop.hive.ql.udf.UDFYear, Args:[%0->$$5], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$29, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$30]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$22, %0->$$28]]]
-                    -- ASSIGN  |PARTITIONED|
-                      project ([$$5, $$29, $$30, $$28, $$48, $$22])
-                      -- STREAM_PROJECT  |PARTITIONED|
+                    exchange 
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      join (function-call: algebricks:eq, Args:[%0->$$24, %0->$$1])
+                      -- HYBRID_HASH_JOIN [$$24][$$1]  |PARTITIONED|
                         exchange 
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          join (function-call: algebricks:eq, Args:[%0->$$24, %0->$$1])
-                          -- HYBRID_HASH_JOIN [$$24][$$1]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
+                          project ([$$29, $$30, $$28, $$24, $$48, $$22])
+                          -- STREAM_PROJECT  |PARTITIONED|
                             exchange 
-                            -- HASH_PARTITION_EXCHANGE [$$24]  |PARTITIONED|
-                              project ([$$29, $$30, $$28, $$24, $$48, $$22])
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              join (function-call: algebricks:eq, Args:[%0->$$25, %0->$$10])
+                              -- HYBRID_HASH_JOIN [$$25][$$10]  |PARTITIONED|
                                 exchange 
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  join (function-call: algebricks:eq, Args:[%0->$$25, %0->$$10])
-                                  -- HYBRID_HASH_JOIN [$$25][$$10]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
+                                  project ([$$22, $$29, $$30, $$28, $$25, $$24, $$48])
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     exchange 
-                                    -- HASH_PARTITION_EXCHANGE [$$25]  |PARTITIONED|
-                                      project ([$$22, $$29, $$30, $$28, $$25, $$24, $$48])
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      join (function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$26, %0->$$20], function-call: algebricks:eq, Args:[%0->$$25, %0->$$19]])
+                                      -- HYBRID_HASH_JOIN [$$26, $$25][$$20, $$19]  |PARTITIONED|
                                         exchange 
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          join (function-call: algebricks:and, Args:[function-call: algebricks:eq, Args:[%0->$$26, %0->$$20], function-call: algebricks:eq, Args:[%0->$$25, %0->$$19]])
-                                          -- HYBRID_HASH_JOIN [$$26, $$25][$$20, $$19]  |PARTITIONED|
+                                          project ([$$48, $$24, $$25, $$26, $$28, $$29, $$30])
+                                          -- STREAM_PROJECT  |PARTITIONED|
                                             exchange 
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              project ([$$48, $$24, $$25, $$26, $$28, $$29, $$30])
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                              join (function-call: algebricks:eq, Args:[%0->$$26, %0->$$40])
+                                              -- HYBRID_HASH_JOIN [$$26][$$40]  |PARTITIONED|
                                                 exchange 
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  join (function-call: algebricks:eq, Args:[%0->$$26, %0->$$40])
-                                                  -- HYBRID_HASH_JOIN [$$26][$$40]  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$26]  |PARTITIONED|
+                                                  data-scan [$$26, $$24, $$25, $$28, $$29, $$30]<-[$$24, $$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33, $$34, $$35, $$36, $$37, $$38, $$39] <- default.lineitem
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
                                                     exchange 
-                                                    -- HASH_PARTITION_EXCHANGE [$$26]  |PARTITIONED|
-                                                      data-scan [$$26, $$24, $$25, $$28, $$29, $$30]<-[$$24, $$25, $$26, $$27, $$28, $$29, $$30, $$31, $$32, $$33, $$34, $$35, $$36, $$37, $$38, $$39] <- default.lineitem
-                                                      -- DATASOURCE_SCAN  |PARTITIONED|
-                                                        exchange 
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          empty-tuple-source
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      empty-tuple-source
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                exchange 
+                                                -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                                                  project ([$$40, $$48])
+                                                  -- STREAM_PROJECT  |PARTITIONED|
                                                     exchange 
-                                                    -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
-                                                      project ([$$40, $$48])
-                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      join (function-call: algebricks:eq, Args:[%0->$$43, %0->$$47])
+                                                      -- HYBRID_HASH_JOIN [$$43][$$47]  |PARTITIONED|
                                                         exchange 
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          join (function-call: algebricks:eq, Args:[%0->$$43, %0->$$47])
-                                                          -- HYBRID_HASH_JOIN [$$43][$$47]  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
+                                                          data-scan [$$43, $$40]<-[$$40, $$41, $$42, $$43, $$44, $$45, $$46] <- default.supplier
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                                             exchange 
-                                                            -- HASH_PARTITION_EXCHANGE [$$43]  |PARTITIONED|
-                                                              data-scan [$$43, $$40]<-[$$40, $$41, $$42, $$43, $$44, $$45, $$46] <- default.supplier
-                                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                exchange 
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  empty-tuple-source
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              empty-tuple-source
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        exchange 
+                                                        -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
+                                                          data-scan [$$47, $$48]<-[$$47, $$48, $$49, $$50] <- default.nation
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                                             exchange 
-                                                            -- HASH_PARTITION_EXCHANGE [$$47]  |PARTITIONED|
-                                                              data-scan [$$47, $$48]<-[$$47, $$48, $$49, $$50] <- default.nation
-                                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                exchange 
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  empty-tuple-source
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              empty-tuple-source
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        exchange 
+                                        -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                          data-scan [$$20, $$19, $$22]<-[$$19, $$20, $$21, $$22, $$23] <- default.partsupp
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                             exchange 
-                                            -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
-                                              data-scan [$$20, $$19, $$22]<-[$$19, $$20, $$21, $$22, $$23] <- default.partsupp
-                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                exchange 
-                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  empty-tuple-source
-                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                    exchange 
-                                    -- HASH_PARTITION_EXCHANGE [$$10]  |PARTITIONED|
-                                      project ([$$10])
-                                      -- STREAM_PROJECT  |PARTITIONED|
-                                        select (function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$11, %green%])
-                                        -- STREAM_SELECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              empty-tuple-source
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                exchange 
+                                -- HASH_PARTITION_EXCHANGE [$$10]  |PARTITIONED|
+                                  project ([$$10])
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    select (function-call: hive:org.apache.hadoop.hive.ql.udf.UDFLike, Args:[%0->$$11, %green%])
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      exchange 
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        data-scan [$$10, $$11]<-[$$10, $$11, $$12, $$13, $$14, $$15, $$16, $$17, $$18] <- default.part
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
                                           exchange 
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            data-scan [$$10, $$11]<-[$$10, $$11, $$12, $$13, $$14, $$15, $$16, $$17, $$18] <- default.part
-                                            -- DATASOURCE_SCAN  |PARTITIONED|
-                                              exchange 
-                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                empty-tuple-source
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            empty-tuple-source
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        exchange 
+                        -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
+                          data-scan [$$1, $$5]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.orders
+                          -- DATASOURCE_SCAN  |PARTITIONED|
                             exchange 
-                            -- HASH_PARTITION_EXCHANGE [$$1]  |PARTITIONED|
-                              data-scan [$$1, $$5]<-[$$1, $$2, $$3, $$4, $$5, $$6, $$7, $$8, $$9] <- default.orders
-                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                exchange 
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  empty-tuple-source
-                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              empty-tuple-source
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u1_group_by.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u1_group_by.plan
index bec1353..3d8c8f7 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u1_group_by.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u1_group_by.plan
@@ -10,16 +10,16 @@
                   nested tuple source
                   -- NESTED_TUPLE_SOURCE  |LOCAL|
              }
-      -- EXTERNAL_GROUP_BY[$$23]  |PARTITIONED|
+      -- PRE_CLUSTERED_GROUP_BY[$$23]  |PARTITIONED|
         exchange 
-        -- HASH_PARTITION_EXCHANGE [$$23]  |PARTITIONED|
+        -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$23(ASC)] HASH:[$$23]  |PARTITIONED|
           group by ([$$23 := %0->$$9]) decor ([]) {
                     aggregate [$$21, $$22] <- [function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFAbs, Args:[%0->$$5]], function-call: hive:sum(PARTIAL1), Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMultiply, Args:[%0->$$6, function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPMinus, Args:[1, %0->$$7]], function-call: hive:org.apache.hadoop.hive.ql.udf.UDFOPPlus, Args:[1, %0->$$8]]]]
                     -- AGGREGATE  |LOCAL|
                       nested tuple source
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                  }
-          -- EXTERNAL_GROUP_BY[$$9]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$9]  |PARTITIONED|
             exchange 
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
               project ([$$9, $$5, $$6, $$7, $$8])
diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan
index b5f1dc2..897f74b 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan
+++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/u7_multi_join.plan
@@ -47,4 +47,4 @@
                           exchange 
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             empty-tuple-source
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
index 49cdedf..e8d87dd 100644
--- a/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
+++ b/hivesterix/hivesterix-dist/src/test/resources/runtimefunctionts/hive/conf/hive-default.xml
@@ -25,7 +25,7 @@
 
 	<property>
 		<name>hive.algebricks.groupby.external</name>
-		<value>true</value>
+		<value>false</value>
 	</property>
 
 	<property>
diff --git a/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java b/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
index 1f31e44..6eb25a7 100644
--- a/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
+++ b/hivesterix/hivesterix-optimizer/src/main/java/edu/uci/ics/hivesterix/optimizer/rulecollections/HiveRuleCollections.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ComplexJoinInferenceRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
@@ -38,7 +39,7 @@
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByCombinerRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.CopyLimitDownRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushGroupByIntoSortRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectDownRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushProjectIntoDataSourceScanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectDownRule;
@@ -121,6 +122,7 @@
         prepareJobGenRules.add(new LocalGroupByRule());
         prepareJobGenRules.add(new PushProjectIntoDataSourceScanRule());
         prepareJobGenRules.add(new ReinferAllTypesRule());
+        prepareJobGenRules.add(new PushGroupByIntoSortRule());
         prepareJobGenRules.add(new SetExecutionModeRule());
     }
 
diff --git a/hyracks/hyracks-api/pom.xml b/hyracks/hyracks-api/pom.xml
index 734d671..99f728f 100644
--- a/hyracks/hyracks-api/pom.xml
+++ b/hyracks/hyracks-api/pom.xml
@@ -47,7 +47,7 @@
   	<dependency>
   		<groupId>org.apache.httpcomponents</groupId>
   		<artifactId>httpclient</artifactId>
-  		<version>4.1-alpha2</version>
+  		<version>4.3</version>
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
index 48d7275..e71f3c9 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
@@ -74,6 +74,7 @@
         acg.setGlobalJobDataFactory(spec.getGlobalJobDataFactory());
         acg.setConnectorPolicyAssignmentPolicy(spec.getConnectorPolicyAssignmentPolicy());
         acg.setUseConnectorPolicyForScheduling(spec.isUseConnectorPolicyForScheduling());
+        acg.setReportTaskDetails(spec.isReportTaskDetails());
         final Set<Constraint> constraints = new HashSet<Constraint>();
         final IConstraintAcceptor acceptor = new IConstraintAcceptor() {
             @Override
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
index e93ebeb..3bb7d22 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NetworkAddress.java
@@ -14,15 +14,30 @@
  */
 package edu.uci.ics.hyracks.api.comm;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.Arrays;
 
-public final class NetworkAddress implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public final class NetworkAddress implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final byte[] ipAddress;
+    private byte[] ipAddress;
 
-    private final int port;
+    private int port;
+
+    public static NetworkAddress create(DataInput dis) throws IOException {
+        NetworkAddress networkAddress = new NetworkAddress();
+        networkAddress.readFields(dis);
+        return networkAddress;
+    }
+
+    private NetworkAddress() {
+
+    }
 
     public NetworkAddress(byte[] ipAddress, int port) {
         this.ipAddress = ipAddress;
@@ -55,4 +70,19 @@
         NetworkAddress on = (NetworkAddress) o;
         return on.port == port && Arrays.equals(on.ipAddress, ipAddress);
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeInt(ipAddress.length);
+        output.write(ipAddress);
+        output.writeInt(port);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        int size = input.readInt();
+        ipAddress = new byte[size];
+        input.readFully(ipAddress);
+        port = input.readInt();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
index af63632..68560e1 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
@@ -14,12 +14,27 @@
  */
 package edu.uci.ics.hyracks.api.dataflow;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
-public final class ActivityId implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public final class ActivityId implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
-    private final OperatorDescriptorId odId;
-    private final int id;
+    private OperatorDescriptorId odId;
+    private int id;
+
+    public static ActivityId create(DataInput dis) throws IOException {
+        ActivityId activityId = new ActivityId();
+        activityId.readFields(dis);
+        return activityId;
+    }
+
+    private ActivityId() {
+
+    }
 
     public ActivityId(OperatorDescriptorId odId, int id) {
         this.odId = odId;
@@ -64,4 +79,16 @@
         }
         throw new IllegalArgumentException("Unable to parse: " + str);
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        odId.writeFields(output);
+        output.writeInt(id);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        odId = OperatorDescriptorId.create(input);
+        id = input.readInt();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
index b363556..5190cae 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
@@ -14,13 +14,28 @@
  */
 package edu.uci.ics.hyracks.api.dataflow;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
-public final class ConnectorDescriptorId implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public final class ConnectorDescriptorId implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
     private int id;
 
+    public static ConnectorDescriptorId create(DataInput dis) throws IOException {
+        ConnectorDescriptorId connectorDescriptorId = new ConnectorDescriptorId();
+        connectorDescriptorId.readFields(dis);
+        return connectorDescriptorId;
+    }
+
+    private ConnectorDescriptorId() {
+
+    }
+
     public ConnectorDescriptorId(int id) {
         this.id = id;
     }
@@ -50,4 +65,14 @@
     public String toString() {
         return "CDID:" + id;
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeInt(id);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        id = input.readInt();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
index 0c23465..5351c78 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
@@ -14,12 +14,27 @@
  */
 package edu.uci.ics.hyracks.api.dataflow;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
-public final class OperatorDescriptorId implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public final class OperatorDescriptorId implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final int id;
+    private int id;
+
+    public static OperatorDescriptorId create(DataInput dis) throws IOException {
+        OperatorDescriptorId operatorDescriptorId = new OperatorDescriptorId();
+        operatorDescriptorId.readFields(dis);
+        return operatorDescriptorId;
+    }
+
+    private OperatorDescriptorId() {
+
+    }
 
     public OperatorDescriptorId(int id) {
         this.id = id;
@@ -57,4 +72,14 @@
         }
         throw new IllegalArgumentException("Unable to parse: " + str);
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeInt(id);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        id = input.readInt();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
index 65fa2e5..2355e98 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
@@ -14,14 +14,29 @@
  */
 package edu.uci.ics.hyracks.api.dataflow;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
-public final class TaskAttemptId implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public final class TaskAttemptId implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final TaskId taskId;
+    private TaskId taskId;
 
-    private final int attempt;
+    private int attempt;
+
+    public static TaskAttemptId create(DataInput dis) throws IOException {
+        TaskAttemptId taskAttemptId = new TaskAttemptId();
+        taskAttemptId.readFields(dis);
+        return taskAttemptId;
+    }
+
+    private TaskAttemptId() {
+
+    }
 
     public TaskAttemptId(TaskId taskId, int attempt) {
         this.taskId = taskId;
@@ -63,4 +78,16 @@
         }
         throw new IllegalArgumentException("Unable to parse: " + str);
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        taskId.writeFields(output);
+        output.writeInt(attempt);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        taskId = TaskId.create(input);
+        attempt = input.readInt();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
index 6d58bd9..6b9eecc 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
@@ -14,14 +14,29 @@
  */
 package edu.uci.ics.hyracks.api.dataflow;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
-public final class TaskId implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public final class TaskId implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final ActivityId activityId;
+    private ActivityId activityId;
 
-    private final int partition;
+    private int partition;
+
+    public static TaskId create(DataInput dis) throws IOException {
+        TaskId taskId = new TaskId();
+        taskId.readFields(dis);
+        return taskId;
+    }
+
+    private TaskId() {
+
+    }
 
     public TaskId(ActivityId activityId, int partition) {
         this.activityId = activityId;
@@ -63,4 +78,16 @@
         }
         throw new IllegalArgumentException("Unable to parse: " + str);
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        activityId.writeFields(output);
+        output.writeInt(partition);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        activityId = ActivityId.create(input);
+        partition = input.readInt();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/ConnectorPolicyFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/ConnectorPolicyFactory.java
new file mode 100644
index 0000000..8b416da
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/ConnectorPolicyFactory.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.hyracks.api.dataflow.connectors;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+/**
+ * @author yingyib
+ */
+public class ConnectorPolicyFactory {
+    public static ConnectorPolicyFactory INSTANCE = new ConnectorPolicyFactory();
+
+    private ConnectorPolicyFactory() {
+
+    }
+
+    public IConnectorPolicy getConnectorPolicy(DataInput input) throws IOException {
+        int kind = input.readInt();
+        switch (kind) {
+            case 0:
+                return new PipeliningConnectorPolicy();
+            case 1:
+                return new SendSideMaterializedBlockingConnectorPolicy();
+            case 2:
+                return new SendSideMaterializedPipeliningConnectorPolicy();
+            case 3:
+                return new SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy();
+            case 4:
+                return new SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy();
+            case 5:
+                return new SendSidePipeliningReceiveSideMaterializedBlockingConnectorPolicy();
+        }
+        return null;
+    }
+
+    public void writeConnectorPolicy(IConnectorPolicy policy, DataOutput output) throws IOException {
+        if (policy instanceof PipeliningConnectorPolicy) {
+            output.writeInt(0);
+        } else if (policy instanceof SendSideMaterializedBlockingConnectorPolicy) {
+            output.writeInt(1);
+        } else if (policy instanceof SendSideMaterializedPipeliningConnectorPolicy) {
+            output.writeInt(2);
+        } else if (policy instanceof SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy) {
+            output.writeInt(3);
+        } else if (policy instanceof SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy) {
+            output.writeInt(4);
+        } else if (policy instanceof SendSidePipeliningReceiveSideMaterializedBlockingConnectorPolicy) {
+            output.writeInt(5);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSidePipeliningReceiveSideMaterializedBlockingConnectorPolicy.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSidePipeliningReceiveSideMaterializedBlockingConnectorPolicy.java
new file mode 100644
index 0000000..8beb2f6
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSidePipeliningReceiveSideMaterializedBlockingConnectorPolicy.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.hyracks.api.dataflow.connectors;
+
+public class SendSidePipeliningReceiveSideMaterializedBlockingConnectorPolicy implements IConnectorPolicy {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public boolean requiresProducerConsumerCoscheduling() {
+        return true;
+    }
+
+    @Override
+    public boolean consumerWaitsForProducerToFinish() {
+        return false;
+    }
+
+    @Override
+    public boolean materializeOnSendSide() {
+        return false;
+    }
+
+    @Override
+    public boolean materializeOnReceiveSide() {
+        return true;
+    }
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java
index 6eab7a7..f461e52 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/deployment/DeploymentId.java
@@ -15,17 +15,32 @@
 
 package edu.uci.ics.hyracks.api.deployment;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.io.IWritable;
+
 /**
  * The representation of a deployment id
  * 
  * @author yingyib
  */
-public class DeploymentId implements Serializable {
+public class DeploymentId implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final String deploymentKey;
+    private String deploymentKey;
+
+    public static DeploymentId create(DataInput dis) throws IOException {
+        DeploymentId deploymentId = new DeploymentId();
+        deploymentId.readFields(dis);
+        return deploymentId;
+    }
+
+    private DeploymentId() {
+
+    }
 
     public DeploymentId(String deploymentKey) {
         this.deploymentKey = deploymentKey;
@@ -50,4 +65,14 @@
     public String toString() {
         return deploymentKey;
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeUTF(deploymentKey);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        deploymentKey = input.readUTF();
+    }
 }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWritable.java
similarity index 67%
copy from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
copy to hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWritable.java
index fb2d1eb..9e7e8c8 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWritable.java
@@ -12,10 +12,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package edu.uci.ics.hyracks.api.io;
 
-package edu.uci.ics.pregelix.dataflow.util;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 
-public enum StorageType {
-    TreeIndex,
-    LSMIndex
+/**
+ * @author yingyib
+ */
+public interface IWritable {
+
+    public void writeFields(DataOutput output) throws IOException;
+
+    public void readFields(DataInput input) throws IOException;
+
 }
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityClusterGraph.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityClusterGraph.java
index e80168b..12c4e6e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityClusterGraph.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityClusterGraph.java
@@ -17,6 +17,7 @@
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.json.JSONArray;
@@ -25,6 +26,7 @@
 
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
 
 public class ActivityClusterGraph implements Serializable {
@@ -50,12 +52,15 @@
 
     private boolean useConnectorPolicyForScheduling;
 
+    private boolean reportTaskDetails;
+
     public ActivityClusterGraph() {
         version = 0;
         activityClusterMap = new HashMap<ActivityClusterId, ActivityCluster>();
         activityMap = new HashMap<ActivityId, ActivityCluster>();
         connectorMap = new HashMap<ConnectorDescriptorId, ActivityCluster>();
         frameSize = 32768;
+        reportTaskDetails = true;
     }
 
     public Map<ActivityId, ActivityCluster> getActivityMap() {
@@ -135,6 +140,24 @@
         this.useConnectorPolicyForScheduling = useConnectorPolicyForScheduling;
     }
 
+    public boolean isReportTaskDetails() {
+        return reportTaskDetails;
+    }
+
+    public void setReportTaskDetails(boolean reportTaskDetails) {
+        this.reportTaskDetails = reportTaskDetails;
+    }
+
+    public List<IConnectorDescriptor> getActivityInputs(ActivityId activityId) {
+        ActivityCluster ac = activityMap.get(activityId);
+        return ac.getActivityInputMap().get(activityId);
+    }
+
+    public ActivityId getProducerActivity(ConnectorDescriptorId cid) {
+        ActivityCluster ac = connectorMap.get(cid);
+        return ac.getProducerActivity(cid);
+    }
+
     public JSONObject toJSON() throws JSONException {
         JSONObject acgj = new JSONObject();
 
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java
index b8eb61b..c9027ba 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java
@@ -14,12 +14,26 @@
  */
 package edu.uci.ics.hyracks.api.job;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
-public final class JobId implements Serializable {
-    private static final long serialVersionUID = 1L;
+import edu.uci.ics.hyracks.api.io.IWritable;
 
-    private final long id;
+public final class JobId implements IWritable, Serializable {
+    private static final long serialVersionUID = 1L;
+    private long id;
+
+    public static JobId create(DataInput dis) throws IOException {
+        JobId jobId = new JobId();
+        jobId.readFields(dis);
+        return jobId;
+    }
+
+    private JobId() {
+
+    }
 
     public JobId(long id) {
         this.id = id;
@@ -57,4 +71,14 @@
         }
         throw new IllegalArgumentException();
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeLong(id);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        id = input.readLong();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 128978b..19904dd 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -75,6 +75,8 @@
 
     private boolean useConnectorPolicyForScheduling;
 
+    private boolean reportTaskDetails;
+
     private transient int operatorIdCounter;
 
     private transient int connectorIdCounter;
@@ -98,7 +100,8 @@
         operatorIdCounter = 0;
         connectorIdCounter = 0;
         maxReattempts = 2;
-        useConnectorPolicyForScheduling = true;
+        useConnectorPolicyForScheduling = false;
+        reportTaskDetails = true;
         setFrameSize(frameSize);
     }
 
@@ -288,6 +291,14 @@
         this.useConnectorPolicyForScheduling = useConnectorPolicyForScheduling;
     }
 
+    public boolean isReportTaskDetails() {
+        return reportTaskDetails;
+    }
+
+    public void setReportTaskDetails(boolean reportTaskDetails) {
+        this.reportTaskDetails = reportTaskDetails;
+    }
+
     private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
         List<V> vList = map.get(key);
         if (vList == null) {
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java
index 2ff71d5..c7e01e6 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java
@@ -14,21 +14,35 @@
  */
 package edu.uci.ics.hyracks.api.partitions;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.io.IWritable;
 import edu.uci.ics.hyracks.api.job.JobId;
 
-public final class PartitionId implements Serializable {
+public final class PartitionId implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final JobId jobId;
+    private JobId jobId;
 
-    private final ConnectorDescriptorId cdId;
+    private ConnectorDescriptorId cdId;
 
-    private final int senderIndex;
+    private int senderIndex;
 
-    private final int receiverIndex;
+    private int receiverIndex;
+
+    public static PartitionId create(DataInput dis) throws IOException {
+        PartitionId partitionId = new PartitionId();
+        partitionId.readFields(dis);
+        return partitionId;
+    }
+
+    private PartitionId() {
+
+    }
 
     public PartitionId(JobId jobId, ConnectorDescriptorId cdId, int senderIndex, int receiverIndex) {
         this.jobId = jobId;
@@ -94,4 +108,20 @@
     public String toString() {
         return jobId.toString() + ":" + cdId + ":" + senderIndex + ":" + receiverIndex;
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        cdId.writeFields(output);
+        jobId.writeFields(output);
+        output.writeInt(receiverIndex);
+        output.writeInt(senderIndex);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        cdId = ConnectorDescriptorId.create(input);
+        jobId = JobId.create(input);
+        receiverIndex = input.readInt();
+        senderIndex = input.readInt();
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/Counters.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/Counters.java
index da30e20..08284cc 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/Counters.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/Counters.java
@@ -22,6 +22,8 @@
 
     public static final String MEMORY_USAGE = "heap-used-sizes";
 
+    public static final String MEMORY_MAX = "heap-max-sizes";
+
     public static final String NETWORK_IO_READ = "net-payload-bytes-read";
 
     public static final String NETWORK_IO_WRITE = "net-payload-bytes-written";
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/impl/ClientCounterContext.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/impl/ClientCounterContext.java
index c39cba7..62bb943 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/impl/ClientCounterContext.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/stats/impl/ClientCounterContext.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.hyracks.client.stats.impl;
 
 import java.io.BufferedReader;
+import java.io.IOException;
 import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
@@ -39,7 +40,7 @@
  */
 public class ClientCounterContext implements IClusterCounterContext {
     private static String[] RESET_COUNTERS = { Counters.NETWORK_IO_READ, Counters.NETWORK_IO_WRITE,
-            Counters.MEMORY_USAGE, Counters.DISK_READ, Counters.DISK_WRITE, Counters.NUM_PROCESSOR };
+            Counters.MEMORY_USAGE, Counters.MEMORY_MAX, Counters.DISK_READ, Counters.DISK_WRITE, Counters.NUM_PROCESSOR };
     private static String[] AGG_COUNTERS = { Counters.SYSTEM_LOAD };
     private static int UPDATE_INTERVAL = 10000;
 
@@ -135,7 +136,7 @@
                 }
             }
         } catch (Exception e) {
-            throw new IllegalStateException(e);
+            //ignore
         }
     }
 
@@ -173,16 +174,24 @@
         } else if (counterObject instanceof JSONArray) {
             JSONArray jArray = (JSONArray) counterObject;
             Object[] values = jArray.toArray();
+            /**
+             * use the last non-zero value as the counter value
+             */
             for (Object value : values) {
                 if (value instanceof Double) {
                     Double dValue = (Double) value;
-                    counterValue += dValue.doubleValue();
+                    double currentVal = dValue.doubleValue();
+                    if (currentVal != 0) {
+                        counterValue = (long) currentVal;
+                    }
                 } else if (value instanceof Long) {
                     Long lValue = (Long) value;
-                    counterValue += lValue.longValue();
+                    long currentVal = lValue.longValue();
+                    if (currentVal != 0) {
+                        counterValue = lValue.longValue();
+                    }
                 }
             }
-            counterValue /= values.length;
         } else {
             Long val = (Long) counterObject;
             counterValue = val.longValue();
@@ -215,7 +224,11 @@
             in.close();
             return response.toString();
         } catch (Exception e) {
-            throw new IllegalStateException(e);
+            if (!(e instanceof java.net.ConnectException || e instanceof IOException)) {
+                throw new IllegalStateException(e);
+            } else {
+                return "";
+            }
         }
     }
 
diff --git a/hyracks/hyracks-client/src/test/java/edu/uci/ics/hyracks/client/stats/ClientCounterContextTest.java b/hyracks/hyracks-client/src/test/java/edu/uci/ics/hyracks/client/stats/ClientCounterContextTest.java
index bbf212f..2ba2631 100644
--- a/hyracks/hyracks-client/src/test/java/edu/uci/ics/hyracks/client/stats/ClientCounterContextTest.java
+++ b/hyracks/hyracks-client/src/test/java/edu/uci/ics/hyracks/client/stats/ClientCounterContextTest.java
@@ -33,8 +33,9 @@
         synchronized (this) {
             wait(20000);
         }
-        String[] counters = { Counters.MEMORY_USAGE, Counters.NETWORK_IO_READ, Counters.NETWORK_IO_WRITE,
-                Counters.SYSTEM_LOAD, Counters.NUM_PROCESSOR, Counters.DISK_READ, Counters.DISK_WRITE };
+        String[] counters = { Counters.MEMORY_USAGE, Counters.MEMORY_MAX, Counters.NETWORK_IO_READ,
+                Counters.NETWORK_IO_WRITE, Counters.SYSTEM_LOAD, Counters.NUM_PROCESSOR, Counters.DISK_READ,
+                Counters.DISK_WRITE };
         for (String counterName : counters) {
             ICounter counter = ccContext.getCounter(counterName, false);
             System.out.println(counterName + ": " + counter.get());
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
index ffb1ace..c5cb7d0 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
@@ -95,9 +95,7 @@
         }
         ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
         ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
-        for (int i = 0; i < nBuffers; ++i) {
-            ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
-        }
+        ccb.getReadInterface().setBufferFactory(new ReadBufferFactory(nBuffers, ctx), nBuffers, ctx.getFrameSize());
         ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
         writeBuffer.putLong(partitionId.getJobId().getId());
         writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
index af1f8f6..b573b73 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
@@ -32,6 +32,10 @@
 
     private boolean aborted;
 
+    private int frameSize = 32768;
+
+    private int allocateCounter = 0;
+
     public NetworkOutputChannel(ChannelControlBlock ccb, int nBuffers) {
         this.ccb = ccb;
         this.nBuffers = nBuffers;
@@ -40,9 +44,7 @@
     }
 
     public void setFrameSize(int frameSize) {
-        for (int i = 0; i < nBuffers; ++i) {
-            emptyStack.push(ByteBuffer.allocateDirect(frameSize));
-        }
+        this.frameSize = frameSize;
     }
 
     @Override
@@ -58,6 +60,10 @@
                     throw new HyracksDataException("Connection has been aborted");
                 }
                 destBuffer = emptyStack.poll();
+                if (destBuffer == null && allocateCounter < nBuffers) {
+                    destBuffer = ByteBuffer.allocateDirect(frameSize);
+                    allocateCounter++;
+                }
                 if (destBuffer != null) {
                     break;
                 }
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
new file mode 100644
index 0000000..c59398c
--- /dev/null
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.comm.channels;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IBufferFactory;
+
+/**
+ * @author yingyib
+ */
+public class ReadBufferFactory implements IBufferFactory {
+
+    private final int limit;
+    private final int frameSize;
+    private int counter = 0;
+
+    public ReadBufferFactory(int limit, IHyracksCommonContext ctx) {
+        this.limit = limit;
+        this.frameSize = ctx.getFrameSize();
+    }
+
+    @Override
+    public ByteBuffer createBuffer() {
+        try {
+            if (counter >= limit) {
+                return null;
+            } else {
+                ByteBuffer frame = ByteBuffer.allocate(frameSize);
+                counter++;
+                return frame;
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index c994dfb..e0bc9e2 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -84,8 +84,8 @@
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 import edu.uci.ics.hyracks.control.common.deployment.DeploymentRun;
 import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions;
-import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions.StateDumpResponseFunction;
 import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions.Function;
+import edu.uci.ics.hyracks.control.common.ipc.CCNCFunctions.StateDumpResponseFunction;
 import edu.uci.ics.hyracks.control.common.logs.LogFile;
 import edu.uci.ics.hyracks.control.common.work.IPCResponder;
 import edu.uci.ics.hyracks.control.common.work.IResultCallback;
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index ad4744b..dbd64a7 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -30,11 +30,13 @@
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.constraints.Constraint;
 import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
 import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
@@ -338,29 +340,45 @@
         tcAttempt.initializePendingTaskCounter();
         tcAttempts.add(tcAttempt);
 
-        /* TODO - Further improvement for reducing messages -- not yet complete.
+        /**
+         * Improvement for reducing master/slave message communications, for each TaskAttemptDescriptor,
+         * we set the NetworkAddress[][] partitionLocations, in which each row is for an incoming connector descriptor
+         * and each column is for an input channel of the connector.
+         */
         for (Map.Entry<String, List<TaskAttemptDescriptor>> e : taskAttemptMap.entrySet()) {
             List<TaskAttemptDescriptor> tads = e.getValue();
             for (TaskAttemptDescriptor tad : tads) {
-                TaskId tid = tad.getTaskAttemptId().getTaskId();
+                TaskAttemptId taid = tad.getTaskAttemptId();
+                int attempt = taid.getAttempt();
+                TaskId tid = taid.getTaskId();
                 ActivityId aid = tid.getActivityId();
-                List<IConnectorDescriptor> inConnectors = jag.getActivityInputConnectorDescriptors(aid);
+                List<IConnectorDescriptor> inConnectors = acg.getActivityInputs(aid);
                 int[] inPartitionCounts = tad.getInputPartitionCounts();
-                NetworkAddress[][] partitionLocations = new NetworkAddress[inPartitionCounts.length][];
-                for (int i = 0; i < inPartitionCounts.length; ++i) {
-                    ConnectorDescriptorId cdId = inConnectors.get(i).getConnectorId();
-                    ActivityId producerAid = jag.getProducerActivity(cdId);
-                    partitionLocations[i] = new NetworkAddress[inPartitionCounts[i]];
-                    for (int j = 0; j < inPartitionCounts[i]; ++j) {
-                        TaskId producerTaskId = new TaskId(producerAid, j);
-                        String nodeId = findTaskLocation(producerTaskId);
-                        partitionLocations[i][j] = ccs.getNodeMap().get(nodeId).getDataPort();
+                if (inPartitionCounts != null) {
+                    NetworkAddress[][] partitionLocations = new NetworkAddress[inPartitionCounts.length][];
+                    for (int i = 0; i < inPartitionCounts.length; ++i) {
+                        ConnectorDescriptorId cdId = inConnectors.get(i).getConnectorId();
+                        IConnectorPolicy policy = jobRun.getConnectorPolicyMap().get(cdId);
+                        /**
+                         * carry sender location information into a task
+                         * when it is not the case that it is an re-attempt and the send-side
+                         * is materialized blocking.
+                         */
+                        if (!(attempt > 0 && policy.materializeOnSendSide() && policy
+                                .consumerWaitsForProducerToFinish())) {
+                            ActivityId producerAid = acg.getProducerActivity(cdId);
+                            partitionLocations[i] = new NetworkAddress[inPartitionCounts[i]];
+                            for (int j = 0; j < inPartitionCounts[i]; ++j) {
+                                TaskId producerTaskId = new TaskId(producerAid, j);
+                                String nodeId = findTaskLocation(producerTaskId);
+                                partitionLocations[i][j] = ccs.getNodeMap().get(nodeId).getDataPort();
+                            }
+                        }
                     }
+                    tad.setInputPartitionLocations(partitionLocations);
                 }
-                tad.setInputPartitionLocations(partitionLocations);
             }
         }
-        */
 
         tcAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.RUNNING);
         tcAttempt.setStartTime(System.currentTimeMillis());
@@ -442,24 +460,25 @@
         final ActivityClusterGraph acg = jobRun.getActivityClusterGraph();
         final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<ConnectorDescriptorId, IConnectorPolicy>(
                 jobRun.getConnectorPolicyMap());
-        for (Map.Entry<String, List<TaskAttemptDescriptor>> entry : taskAttemptMap.entrySet()) {
-            String nodeId = entry.getKey();
-            final List<TaskAttemptDescriptor> taskDescriptors = entry.getValue();
-            final NodeControllerState node = ccs.getNodeMap().get(nodeId);
-            if (node != null) {
-                node.getActiveJobIds().add(jobRun.getJobId());
-                boolean changed = jobRun.getParticipatingNodeIds().add(nodeId);
-                if (LOGGER.isLoggable(Level.FINE)) {
-                    LOGGER.fine("Starting: " + taskDescriptors + " at " + entry.getKey());
-                }
-                try {
-                    byte[] jagBytes = changed ? JavaSerializationUtils.serialize(acg) : null;
+        try {
+            byte[] acgBytes = JavaSerializationUtils.serialize(acg);
+            for (Map.Entry<String, List<TaskAttemptDescriptor>> entry : taskAttemptMap.entrySet()) {
+                String nodeId = entry.getKey();
+                final List<TaskAttemptDescriptor> taskDescriptors = entry.getValue();
+                final NodeControllerState node = ccs.getNodeMap().get(nodeId);
+                if (node != null) {
+                    node.getActiveJobIds().add(jobRun.getJobId());
+                    boolean changed = jobRun.getParticipatingNodeIds().add(nodeId);
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine("Starting: " + taskDescriptors + " at " + entry.getKey());
+                    }
+                    byte[] jagBytes = changed ? acgBytes : null;
                     node.getNodeController().startTasks(deploymentId, jobId, jagBytes, taskDescriptors,
                             connectorPolicies, jobRun.getFlags());
-                } catch (Exception e) {
-                    e.printStackTrace();
                 }
             }
+        } catch (Exception e) {
+            throw new HyracksException(e);
         }
     }
 
@@ -702,10 +721,16 @@
         ccs.getActiveRunMap().remove(jobId);
         ccs.getRunMapArchive().put(jobId, run);
         ccs.getRunHistory().put(jobId, run.getExceptions());
-        try {
-            ccs.getJobLogFile().log(createJobLogObject(run));
-        } catch (Exception e) {
-            throw new RuntimeException(e);
+
+        if (run.getActivityClusterGraph().isReportTaskDetails()) {
+            /**
+             * log job details when task-profiling is enabled
+             */
+            try {
+                ccs.getJobLogFile().log(createJobLogObject(run));
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
         }
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 340134e..5c9aad5 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -105,10 +105,16 @@
         ccs.getActiveRunMap().remove(jobId);
         ccs.getRunMapArchive().put(jobId, run);
         ccs.getRunHistory().put(jobId, run.getExceptions());
-        try {
-            ccs.getJobLogFile().log(createJobLogObject(run));
-        } catch (Exception e) {
-            throw new RuntimeException(e);
+
+        if (run.getActivityClusterGraph().isReportTaskDetails()) {
+            /**
+             * log job details when profiling is enabled
+             */
+            try {
+                ccs.getJobLogFile().log(createJobLogObject(run));
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
         }
     }
 
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index 2d6bdea..9f1c4b2 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -18,11 +18,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.json.JSONException;
-import org.json.JSONObject;
-
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.cc.NodeControllerState;
@@ -71,24 +67,6 @@
             ccs.getActiveRunMap().remove(jobId);
             ccs.getRunMapArchive().put(jobId, run);
             ccs.getRunHistory().put(jobId, run.getExceptions());
-            try {
-                ccs.getJobLogFile().log(createJobLogObject(run));
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-
         }
     }
-
-    private JSONObject createJobLogObject(final JobRun run) {
-        JSONObject jobLogObject = new JSONObject();
-        try {
-            ActivityClusterGraph acg = run.getActivityClusterGraph();
-            jobLogObject.put("activity-cluster-graph", acg.toJSON());
-            jobLogObject.put("job-run", run.toJSON());
-        } catch (JSONException e) {
-            throw new RuntimeException(e);
-        }
-        return jobLogObject;
-    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 01525e4..74e9710 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -49,6 +49,9 @@
     @Option(name = "-net-thread-count", usage = "Number of threads to use for Network I/O (default: 1)")
     public int nNetThreads = 1;
 
+    @Option(name = "-net-buffer-count", usage = "Number of network buffers per input/output channel (default:1)", required = false)
+    public int nNetBuffers = 1;
+
     @Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
     public int maxMemory = -1;
 
@@ -84,6 +87,8 @@
         cList.add(ioDevices);
         cList.add("-net-thread-count");
         cList.add(String.valueOf(nNetThreads));
+        cList.add("-net-buffer-count");
+        cList.add(String.valueOf(nNetBuffers));
         cList.add("-max-memory");
         cList.add(String.valueOf(maxMemory));
         cList.add("-result-time-to-live");
@@ -113,6 +118,7 @@
         configuration.put("data-ip-address", dataIPAddress);
         configuration.put("iodevices", ioDevices);
         configuration.put("net-thread-count", String.valueOf(nNetThreads));
+        configuration.put("net-buffer-count", String.valueOf(nNetBuffers));
         configuration.put("max-memory", String.valueOf(maxMemory));
         configuration.put("result-time-to-live", String.valueOf(resultTTL));
         configuration.put("result-sweep-threshold", String.valueOf(resultSweepThreshold));
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 2407c10..a39a159 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
@@ -28,7 +28,7 @@
 import org.apache.http.HttpResponse;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
-import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
 
 import edu.uci.ics.hyracks.api.application.IApplicationContext;
 import edu.uci.ics.hyracks.api.deployment.DeploymentId;
@@ -200,7 +200,7 @@
                     String filePath = deploymentDir + File.separator + fileName;
                     File targetFile = new File(filePath);
                     if (isNC) {
-                        HttpClient hc = new DefaultHttpClient();
+                        HttpClient hc = HttpClientBuilder.create().build();
                         HttpGet get = new HttpGet(url.toString());
                         HttpResponse response = hc.execute(get);
                         InputStream is = response.getEntity().getContent();
@@ -216,6 +216,7 @@
                 }
                 return downloadedFileURLs;
             } catch (Exception e) {
+                e.printStackTrace();
                 trace = e;
             }
         }
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
index e999913..d574435 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -14,10 +14,11 @@
  */
 package edu.uci.ics.hyracks.control.common.heartbeat;
 
-import java.io.Serializable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 
-public class HeartbeatData implements Serializable {
-    private static final long serialVersionUID = 1L;
+public class HeartbeatData {
 
     public long heapInitSize;
     public long heapUsedSize;
@@ -47,4 +48,83 @@
     public long ipcMessageBytesReceived;
     public long diskReads;
     public long diskWrites;
+
+    public void readFields(DataInput dis) throws IOException {
+        heapInitSize = dis.readLong();
+        heapUsedSize = dis.readLong();
+        heapCommittedSize = dis.readLong();
+        heapMaxSize = dis.readLong();
+        nonheapInitSize = dis.readLong();
+        nonheapUsedSize = dis.readLong();
+        nonheapCommittedSize = dis.readLong();
+        nonheapMaxSize = dis.readLong();
+        threadCount = dis.readInt();
+        peakThreadCount = dis.readInt();
+        totalStartedThreadCount = dis.readLong();
+        systemLoadAverage = dis.readDouble();
+        netPayloadBytesRead = dis.readLong();
+        netPayloadBytesWritten = dis.readLong();
+        netSignalingBytesRead = dis.readLong();
+        netSignalingBytesWritten = dis.readLong();
+        netSignalingBytesWritten = dis.readLong();
+        datasetNetPayloadBytesWritten = dis.readLong();
+        datasetNetSignalingBytesRead = dis.readLong();
+        datasetNetSignalingBytesWritten = dis.readLong();
+        ipcMessagesSent = dis.readLong();
+        ipcMessageBytesSent = dis.readLong();
+        ipcMessagesReceived = dis.readLong();
+        ipcMessageBytesReceived = dis.readLong();
+        diskReads = dis.readLong();
+        diskWrites = dis.readLong();
+
+        int gcCounts = dis.readInt();
+        gcCollectionCounts = new long[gcCounts];
+        for (int i = 0; i < gcCollectionCounts.length; i++) {
+            gcCollectionCounts[i] = dis.readLong();
+        }
+        int gcTimeCounts = dis.readInt();
+        gcCollectionTimes = new long[gcTimeCounts];
+        for (int i = 0; i < gcCollectionTimes.length; i++) {
+            gcCollectionTimes[i] = dis.readLong();
+        }
+    }
+
+    public void write(DataOutput dos) throws IOException {
+        dos.writeLong(heapInitSize);
+        dos.writeLong(heapUsedSize);
+        dos.writeLong(heapCommittedSize);
+        dos.writeLong(heapMaxSize);
+        dos.writeLong(nonheapInitSize);
+        dos.writeLong(nonheapUsedSize);
+        dos.writeLong(nonheapCommittedSize);
+        dos.writeLong(nonheapMaxSize);
+        dos.writeInt(threadCount);
+        dos.writeInt(peakThreadCount);
+        dos.writeLong(totalStartedThreadCount);
+        dos.writeDouble(systemLoadAverage);
+        dos.writeLong(netPayloadBytesRead);
+        dos.writeLong(netPayloadBytesWritten);
+        dos.writeLong(netSignalingBytesRead);
+        dos.writeLong(netSignalingBytesWritten);
+        dos.writeLong(datasetNetPayloadBytesRead);
+        dos.writeLong(datasetNetPayloadBytesWritten);
+        dos.writeLong(datasetNetSignalingBytesRead);
+        dos.writeLong(datasetNetSignalingBytesWritten);
+        dos.writeLong(ipcMessagesSent);
+        dos.writeLong(ipcMessageBytesSent);
+        dos.writeLong(ipcMessagesReceived);
+        dos.writeLong(ipcMessageBytesReceived);
+        dos.writeLong(diskReads);
+        dos.writeLong(diskWrites);
+
+        dos.writeInt(gcCollectionCounts.length);
+        for (int i = 0; i < gcCollectionCounts.length; i++) {
+            dos.writeLong(gcCollectionCounts[i]);
+        }
+        dos.writeInt(gcCollectionTimes.length);
+        for (int i = 0; i < gcCollectionTimes.length; i++) {
+            dos.writeLong(gcCollectionTimes[i]);
+        }
+    }
+
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index 6be2294..1417df9 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -23,9 +23,12 @@
 import java.io.Serializable;
 import java.net.URL;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -36,6 +39,7 @@
 import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.ConnectorPolicyFactory;
 import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.deployment.DeploymentId;
@@ -176,10 +180,10 @@
     public static class NotifyTaskCompleteFunction extends Function {
         private static final long serialVersionUID = 1L;
 
-        private final JobId jobId;
-        private final TaskAttemptId taskId;
-        private final String nodeId;
-        private final TaskProfile statistics;
+        private JobId jobId;
+        private TaskAttemptId taskId;
+        private String nodeId;
+        private TaskProfile statistics;
 
         public NotifyTaskCompleteFunction(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics) {
             this.jobId = jobId;
@@ -208,6 +212,26 @@
         public TaskProfile getStatistics() {
             return statistics;
         }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            JobId jobId = JobId.create(dis);
+            String nodeId = dis.readUTF();
+            TaskAttemptId taskId = TaskAttemptId.create(dis);
+            TaskProfile statistics = TaskProfile.create(dis);
+            return new NotifyTaskCompleteFunction(jobId, taskId, nodeId, statistics);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            NotifyTaskCompleteFunction fn = (NotifyTaskCompleteFunction) object;
+            DataOutputStream dos = new DataOutputStream(out);
+            fn.jobId.writeFields(dos);
+            dos.writeUTF(fn.nodeId);
+            fn.taskId.writeFields(dos);
+            fn.statistics.writeFields(dos);
+        }
     }
 
     public static class NotifyTaskFailureFunction extends Function {
@@ -270,6 +294,23 @@
         public String getNodeId() {
             return nodeId;
         }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            JobId jobId = JobId.create(dis);
+            String nodeId = dis.readUTF();
+
+            return new NotifyJobletCleanupFunction(jobId, nodeId);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            NotifyJobletCleanupFunction fn = (NotifyJobletCleanupFunction) object;
+            DataOutputStream dos = new DataOutputStream(out);
+            fn.jobId.writeFields(dos);
+            dos.writeUTF(fn.nodeId);
+        }
     }
 
     public static class NodeHeartbeatFunction extends Function {
@@ -295,6 +336,23 @@
         public HeartbeatData getHeartbeatData() {
             return hbData;
         }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            String nodeId = dis.readUTF();
+            HeartbeatData hbData = new HeartbeatData();
+            hbData.readFields(dis);
+            return new NodeHeartbeatFunction(nodeId, hbData);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            NodeHeartbeatFunction fn = (NodeHeartbeatFunction) object;
+            DataOutputStream dos = new DataOutputStream(out);
+            dos.writeUTF(fn.nodeId);
+            fn.hbData.write(dos);
+        }
     }
 
     public static class ReportProfileFunction extends Function {
@@ -650,6 +708,90 @@
         public EnumSet<JobFlag> getFlags() {
             return flags;
         }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            //read jobId and taskId
+            JobId jobId = JobId.create(dis);
+            DeploymentId deploymentId = null;
+            boolean hasDeployed = dis.readBoolean();
+            if (hasDeployed) {
+                deploymentId = DeploymentId.create(dis);
+            }
+
+            // read plan bytes
+            int planBytesSize = dis.readInt();
+            byte[] planBytes = null;
+            if (planBytesSize >= 0) {
+                planBytes = new byte[planBytesSize];
+                dis.read(planBytes, 0, planBytesSize);
+            }
+
+            // read task attempt descriptors
+            int tadSize = dis.readInt();
+            List<TaskAttemptDescriptor> taskDescriptors = new ArrayList<TaskAttemptDescriptor>();
+            for (int i = 0; i < tadSize; i++) {
+                TaskAttemptDescriptor tad = TaskAttemptDescriptor.create(dis);
+                taskDescriptors.add(tad);
+            }
+
+            //read connector policies
+            int cpSize = dis.readInt();
+            Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
+            for (int i = 0; i < cpSize; i++) {
+                ConnectorDescriptorId cid = ConnectorDescriptorId.create(dis);
+                IConnectorPolicy policy = ConnectorPolicyFactory.INSTANCE.getConnectorPolicy(dis);
+                connectorPolicies.put(cid, policy);
+            }
+
+            // read flags
+            int flagSize = dis.readInt();
+            EnumSet<JobFlag> flags = EnumSet.noneOf(JobFlag.class);
+            for (int i = 0; i < flagSize; i++) {
+                flags.add(JobFlag.values()[(dis.readInt())]);
+            }
+
+            return new StartTasksFunction(deploymentId, jobId, planBytes, taskDescriptors, connectorPolicies, flags);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            StartTasksFunction fn = (StartTasksFunction) object;
+            DataOutputStream dos = new DataOutputStream(out);
+
+            //write jobId and deploymentId
+            fn.jobId.writeFields(dos);
+            dos.writeBoolean(fn.deploymentId == null ? false : true);
+            if (fn.deploymentId != null) {
+                fn.deploymentId.writeFields(dos);
+            }
+
+            //write plan bytes
+            dos.writeInt(fn.planBytes == null ? -1 : fn.planBytes.length);
+            if (fn.planBytes != null) {
+                dos.write(fn.planBytes, 0, fn.planBytes.length);
+            }
+
+            //write task descriptors
+            dos.writeInt(fn.taskDescriptors.size());
+            for (int i = 0; i < fn.taskDescriptors.size(); i++) {
+                fn.taskDescriptors.get(i).writeFields(dos);
+            }
+
+            //write connector policies
+            dos.writeInt(fn.connectorPolicies.size());
+            for (Entry<ConnectorDescriptorId, IConnectorPolicy> entry : fn.connectorPolicies.entrySet()) {
+                entry.getKey().writeFields(dos);
+                ConnectorPolicyFactory.INSTANCE.writeConnectorPolicy(entry.getValue(), dos);
+            }
+
+            //write flags
+            dos.writeInt(fn.flags.size());
+            for (JobFlag flag : fn.flags) {
+                dos.writeInt(flag.ordinal());
+            }
+        }
     }
 
     public static class AbortTasksFunction extends Function {
@@ -700,6 +842,23 @@
         public JobStatus getStatus() {
             return status;
         }
+
+        public static Object deserialize(ByteBuffer buffer, int length) throws Exception {
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), length);
+            DataInputStream dis = new DataInputStream(bais);
+
+            JobId jobId = JobId.create(dis);
+            JobStatus status = JobStatus.values()[dis.readInt()];
+
+            return new CleanupJobletFunction(jobId, status);
+        }
+
+        public static void serialize(OutputStream out, Object object) throws Exception {
+            CleanupJobletFunction fn = (CleanupJobletFunction) object;
+            DataOutputStream dos = new DataOutputStream(out);
+            fn.jobId.writeFields(dos);
+            dos.writeInt(fn.status.ordinal());
+        }
     }
 
     public static class GetNodeControllersInfoFunction extends Function {
@@ -978,6 +1137,25 @@
                 case REPORT_PARTITION_AVAILABILITY:
                     ReportPartitionAvailabilityFunction.serialize(out, object);
                     return;
+
+                case NODE_HEARTBEAT:
+                    NodeHeartbeatFunction.serialize(out, object);
+                    return;
+
+                case START_TASKS:
+                    StartTasksFunction.serialize(out, object);
+                    return;
+
+                case NOTIFY_TASK_COMPLETE:
+                    NotifyTaskCompleteFunction.serialize(out, object);
+                    return;
+
+                case NOTIFY_JOBLET_CLEANUP:
+                    NotifyJobletCleanupFunction.serialize(out, object);
+                    return;
+                case CLEANUP_JOBLET:
+                    CleanupJobletFunction.serialize(out, object);
+                    return;
             }
             JavaSerializationBasedPayloadSerializerDeserializer.serialize(out, object);
         }
@@ -992,6 +1170,21 @@
 
                 case REPORT_PARTITION_AVAILABILITY:
                     return ReportPartitionAvailabilityFunction.deserialize(buffer, length);
+
+                case NODE_HEARTBEAT:
+                    return NodeHeartbeatFunction.deserialize(buffer, length);
+
+                case START_TASKS:
+                    return StartTasksFunction.deserialize(buffer, length);
+
+                case NOTIFY_TASK_COMPLETE:
+                    return NotifyTaskCompleteFunction.deserialize(buffer, length);
+
+                case NOTIFY_JOBLET_CLEANUP:
+                    return NotifyJobletCleanupFunction.deserialize(buffer, length);
+
+                case CLEANUP_JOBLET:
+                    return CleanupJobletFunction.deserialize(buffer, length);
             }
 
             return javaSerde.deserializeObject(buffer, length);
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java
index 6018132..43b39d3 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java
@@ -14,25 +14,39 @@
  */
 package edu.uci.ics.hyracks.control.common.job;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.Arrays;
 
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.io.IWritable;
 
-public class TaskAttemptDescriptor implements Serializable {
+public class TaskAttemptDescriptor implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final TaskAttemptId taId;
+    private TaskAttemptId taId;
 
-    private final int nPartitions;
+    private int nPartitions;
 
-    private final int[] nInputPartitions;
+    private int[] nInputPartitions;
 
-    private final int[] nOutputPartitions;
+    private int[] nOutputPartitions;
 
     private NetworkAddress[][] inputPartitionLocations;
 
+    public static TaskAttemptDescriptor create(DataInput dis) throws IOException {
+        TaskAttemptDescriptor taskAttemptDescriptor = new TaskAttemptDescriptor();
+        taskAttemptDescriptor.readFields(dis);
+        return taskAttemptDescriptor;
+    }
+
+    private TaskAttemptDescriptor() {
+
+    }
+
     public TaskAttemptDescriptor(TaskAttemptId taId, int nPartitions, int[] nInputPartitions, int[] nOutputPartitions) {
         this.taId = taId;
         this.nPartitions = nPartitions;
@@ -70,4 +84,74 @@
                 + Arrays.toString(nInputPartitions) + ", nOutputPartitions = " + Arrays.toString(nOutputPartitions)
                 + "]";
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        taId.writeFields(output);
+        output.writeInt(nPartitions);
+
+        output.writeInt(nInputPartitions == null ? -1 : nInputPartitions.length);
+        if (nInputPartitions != null) {
+            for (int i = 0; i < nInputPartitions.length; i++) {
+                output.writeInt(nInputPartitions[i]);
+            }
+        }
+
+        output.writeInt(nOutputPartitions == null ? -1 : nOutputPartitions.length);
+        if (nOutputPartitions != null) {
+            for (int i = 0; i < nOutputPartitions.length; i++) {
+                output.writeInt(nOutputPartitions[i]);
+            }
+        }
+
+        output.writeInt(inputPartitionLocations == null ? -1 : inputPartitionLocations.length);
+        if (inputPartitionLocations != null) {
+            for (int i = 0; i < inputPartitionLocations.length; i++) {
+                if (inputPartitionLocations[i] != null) {
+                    output.writeInt(inputPartitionLocations[i].length);
+                    for (int j = 0; j < inputPartitionLocations[i].length; j++) {
+                        inputPartitionLocations[i][j].writeFields(output);
+                    }
+                } else {
+                    output.writeInt(-1);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        taId = TaskAttemptId.create(input);
+        nPartitions = input.readInt();
+
+        int inputCount = input.readInt();
+        if (inputCount >= 0) {
+            nInputPartitions = new int[inputCount];
+            for (int i = 0; i < nInputPartitions.length; i++) {
+                nInputPartitions[i] = input.readInt();
+            }
+        }
+
+        int outputCount = input.readInt();
+        if (outputCount >= 0) {
+            nOutputPartitions = new int[outputCount];
+            for (int i = 0; i < nOutputPartitions.length; i++) {
+                nOutputPartitions[i] = input.readInt();
+            }
+        }
+
+        int addrCount = input.readInt();
+        if (addrCount >= 0) {
+            inputPartitionLocations = new NetworkAddress[addrCount][];
+            for (int i = 0; i < inputPartitionLocations.length; i++) {
+                int columns = input.readInt();
+                if (columns >= 0) {
+                    inputPartitionLocations[i] = new NetworkAddress[columns];
+                    for (int j = 0; j < columns; j++) {
+                        inputPartitionLocations[i][j] = NetworkAddress.create(input);
+                    }
+                }
+            }
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/counters/MultiResolutionEventProfiler.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/counters/MultiResolutionEventProfiler.java
index 2080718..b8b90cb 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/counters/MultiResolutionEventProfiler.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/counters/MultiResolutionEventProfiler.java
@@ -14,12 +14,17 @@
  */
 package edu.uci.ics.hyracks.control.common.job.profiling.counters;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
-public class MultiResolutionEventProfiler implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public class MultiResolutionEventProfiler implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final int[] times;
+    private int[] times;
 
     private long offset;
 
@@ -29,6 +34,16 @@
 
     private int eventCounter;
 
+    public static MultiResolutionEventProfiler create(DataInput dis) throws IOException {
+        MultiResolutionEventProfiler multiResolutionEventProfiler = new MultiResolutionEventProfiler();
+        multiResolutionEventProfiler.readFields(dis);
+        return multiResolutionEventProfiler;
+    }
+
+    private MultiResolutionEventProfiler() {
+
+    }
+
     public MultiResolutionEventProfiler(int nSamples) {
         times = new int[nSamples];
         offset = -1;
@@ -78,4 +93,29 @@
     public long getOffset() {
         return offset;
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeInt(eventCounter);
+        output.writeLong(offset);
+        output.writeInt(ptr);
+        output.writeInt(resolution);
+        output.writeInt(times.length);
+        for (int i = 0; i < times.length; i++) {
+            output.writeInt(times[i]);
+        }
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        eventCounter = input.readInt();
+        offset = input.readLong();
+        ptr = input.readInt();
+        resolution = input.readInt();
+        int nSamples = input.readInt();
+        times = new int[nSamples];
+        for (int i = 0; i < times.length; i++) {
+            times[i] = input.readInt();
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
index 2cb4191..12cd4b1 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
@@ -14,18 +14,24 @@
  */
 package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
-public abstract class AbstractProfile implements Serializable {
+import edu.uci.ics.hyracks.api.io.IWritable;
+
+public abstract class AbstractProfile implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    protected final Map<String, Long> counters;
+    protected Map<String, Long> counters;
 
     public AbstractProfile() {
         counters = new HashMap<String, Long>();
@@ -51,4 +57,24 @@
     protected void merge(AbstractProfile profile) {
         counters.putAll(profile.counters);
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeInt(counters.size());
+        for (Entry<String, Long> entry : counters.entrySet()) {
+            output.writeUTF(entry.getKey());
+            output.writeLong(entry.getValue());
+        }
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        int size = input.readInt();
+        counters = new HashMap<String, Long>();
+        for (int i = 0; i < size; i++) {
+            String key = input.readUTF();
+            long value = input.readLong();
+            counters.put(key, value);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
index a941187..a3f7e41 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -14,8 +14,12 @@
  */
 package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
+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.json.JSONArray;
 import org.json.JSONException;
@@ -26,9 +30,19 @@
 public class JobProfile extends AbstractProfile {
     private static final long serialVersionUID = 1L;
 
-    private final JobId jobId;
+    private JobId jobId;
 
-    private final Map<String, JobletProfile> jobletProfiles;
+    private Map<String, JobletProfile> jobletProfiles;
+
+    public static JobProfile create(DataInput dis) throws IOException {
+        JobProfile jobProfile = new JobProfile();
+        jobProfile.readFields(dis);
+        return jobProfile;
+    }
+
+    private JobProfile() {
+
+    }
 
     public JobProfile(JobId jobId) {
         this.jobId = jobId;
@@ -68,4 +82,26 @@
             }
         }
     }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        jobId = JobId.create(input);
+        int size = input.readInt();
+        jobletProfiles = new HashMap<String, JobletProfile>();
+        for (int i = 0; i < size; i++) {
+            String key = input.readUTF();
+            JobletProfile value = JobletProfile.create(input);
+            jobletProfiles.put(key, value);
+        }
+    }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        jobId.writeFields(output);
+        output.writeInt(jobletProfiles.size());
+        for (Entry<String, JobletProfile> entry : jobletProfiles.entrySet()) {
+            output.writeUTF(entry.getKey());
+            entry.getValue().writeFields(output);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
index 16d08d7..a879873 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
@@ -14,8 +14,12 @@
  */
 package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
+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.json.JSONArray;
 import org.json.JSONException;
@@ -26,9 +30,19 @@
 public class JobletProfile extends AbstractProfile {
     private static final long serialVersionUID = 1L;
 
-    private final String nodeId;
+    private String nodeId;
 
-    private final Map<TaskAttemptId, TaskProfile> taskProfiles;
+    private Map<TaskAttemptId, TaskProfile> taskProfiles;
+
+    public static JobletProfile create(DataInput dis) throws IOException {
+        JobletProfile jobletProfile = new JobletProfile();
+        jobletProfile.readFields(dis);
+        return jobletProfile;
+    }
+
+    private JobletProfile() {
+
+    }
 
     public JobletProfile(String nodeId) {
         this.nodeId = nodeId;
@@ -68,4 +82,28 @@
             }
         }
     }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        nodeId = input.readUTF();
+        int size = input.readInt();
+        taskProfiles = new HashMap<TaskAttemptId, TaskProfile>();
+        for (int i = 0; i < size; i++) {
+            TaskAttemptId key = TaskAttemptId.create(input);
+            TaskProfile value = TaskProfile.create(input);
+            taskProfiles.put(key, value);
+        }
+    }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        super.writeFields(output);
+        output.writeUTF(nodeId);
+        output.writeInt(taskProfiles.size());
+        for (Entry<TaskAttemptId, TaskProfile> entry : taskProfiles.entrySet()) {
+            entry.getKey().writeFields(output);
+            entry.getValue().writeFields(output);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
index a9cc979..3fc456d 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/PartitionProfile.java
@@ -14,21 +14,35 @@
  */
 package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
+import edu.uci.ics.hyracks.api.io.IWritable;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
 import edu.uci.ics.hyracks.control.common.job.profiling.counters.MultiResolutionEventProfiler;
 
-public class PartitionProfile implements Serializable {
+public class PartitionProfile implements IWritable, Serializable {
     private static final long serialVersionUID = 1L;
 
-    private final PartitionId pid;
+    private PartitionId pid;
 
-    private final long openTime;
+    private long openTime;
 
-    private final long closeTime;
+    private long closeTime;
 
-    private final MultiResolutionEventProfiler mrep;
+    private MultiResolutionEventProfiler mrep;
+
+    public static PartitionProfile create(DataInput dis) throws IOException {
+        PartitionProfile partitionProfile = new PartitionProfile();
+        partitionProfile.readFields(dis);
+        return partitionProfile;
+    }
+
+    private PartitionProfile() {
+
+    }
 
     public PartitionProfile(PartitionId pid, long openTime, long closeTime, MultiResolutionEventProfiler mrep) {
         this.pid = pid;
@@ -52,4 +66,20 @@
     public MultiResolutionEventProfiler getSamples() {
         return mrep;
     }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        output.writeLong(closeTime);
+        output.writeLong(openTime);
+        mrep.writeFields(output);
+        pid.writeFields(output);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        closeTime = input.readLong();
+        openTime = input.readLong();
+        mrep = MultiResolutionEventProfiler.create(input);
+        pid = PartitionId.create(input);
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
index 6918af4..8774a50 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -14,8 +14,12 @@
  */
 package edu.uci.ics.hyracks.control.common.job.profiling.om;
 
+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.json.JSONArray;
 import org.json.JSONException;
@@ -28,9 +32,19 @@
 public class TaskProfile extends AbstractProfile {
     private static final long serialVersionUID = 1L;
 
-    private final TaskAttemptId taskAttemptId;
+    private TaskAttemptId taskAttemptId;
 
-    private final Map<PartitionId, PartitionProfile> partitionSendProfile;
+    private Map<PartitionId, PartitionProfile> partitionSendProfile;
+
+    public static TaskProfile create(DataInput dis) throws IOException {
+        TaskProfile taskProfile = new TaskProfile();
+        taskProfile.readFields(dis);
+        return taskProfile;
+    }
+
+    private TaskProfile() {
+
+    }
 
     public TaskProfile(TaskAttemptId taskAttemptId, Map<PartitionId, PartitionProfile> partitionSendProfile) {
         this.taskAttemptId = taskAttemptId;
@@ -84,4 +98,28 @@
 
         return json;
     }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        taskAttemptId = TaskAttemptId.create(input);
+        int size = input.readInt();
+        partitionSendProfile = new HashMap<PartitionId, PartitionProfile>();
+        for (int i = 0; i < size; i++) {
+            PartitionId key = PartitionId.create(input);
+            PartitionProfile value = PartitionProfile.create(input);
+            partitionSendProfile.put(key, value);
+        }
+    }
+
+    @Override
+    public void writeFields(DataOutput output) throws IOException {
+        super.writeFields(output);
+        taskAttemptId.writeFields(output);
+        output.writeInt(partitionSendProfile.size());
+        for (Entry<PartitionId, PartitionProfile> entry : partitionSendProfile.entrySet()) {
+            entry.getKey().writeFields(output);
+            entry.getValue().writeFields(output);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 56b6654..89c5b75 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -267,6 +267,10 @@
         return globalJobData;
     }
 
+    public IJobletEventListener getJobletEventListener() {
+        return jobletEventListener;
+    }
+
     public synchronized void advertisePartitionRequest(TaskAttemptId taId, Collection<PartitionId> pids,
             IPartitionCollector collector, PartitionState minState) throws Exception {
         for (PartitionId pid : pids) {
@@ -283,10 +287,6 @@
         }
     }
 
-    public IJobletEventListener getJobletEventListener() {
-        return jobletEventListener;
-    }
-
     public void cleanup(JobStatus status) {
         cleanupStatus = status;
         cleanupPending = true;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index f0b570e..5eec7bb 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -168,7 +168,8 @@
             throw new Exception("id not set");
         }
         partitionManager = new PartitionManager(this);
-        netManager = new NetworkManager(getIpAddress(ncConfig.dataIPAddress), partitionManager, ncConfig.nNetThreads);
+        netManager = new NetworkManager(getIpAddress(ncConfig.dataIPAddress), partitionManager, ncConfig.nNetThreads,
+                ncConfig.nNetBuffers);
 
         lccm = new LifeCycleComponentManager();
         queue = new WorkQueue();
@@ -243,7 +244,7 @@
         datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory,
                 ncConfig.resultTTL, ncConfig.resultSweepThreshold);
         datasetNetworkManager = new DatasetNetworkManager(getIpAddress(ncConfig.datasetIPAddress),
-                datasetPartitionManager, ncConfig.nNetThreads);
+                datasetPartitionManager, ncConfig.nNetThreads, ncConfig.nNetBuffers);
     }
 
     @Override
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 fa9b6b3..3014024 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
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -89,7 +90,10 @@
 
     private NodeControllerService ncs;
 
-    public Task(Joblet joblet, TaskAttemptId taskId, String displayName, Executor executor, NodeControllerService ncs) {
+    private List<List<PartitionChannel>> inputChannelsFromConnectors;
+
+    public Task(Joblet joblet, TaskAttemptId taskId, String displayName, Executor executor, NodeControllerService ncs,
+            List<List<PartitionChannel>> inputChannelsFromConnectors) {
         this.joblet = joblet;
         this.taskAttemptId = taskId;
         this.displayName = displayName;
@@ -102,6 +106,7 @@
         pendingThreads = new LinkedHashSet<Thread>();
         exceptions = new ArrayList<>();
         this.ncs = ncs;
+        this.inputChannelsFromConnectors = inputChannelsFromConnectors;
     }
 
     public void setTaskRuntime(IPartitionCollector[] collectors, IOperatorNodePushable operator) {
@@ -113,7 +118,7 @@
     public ByteBuffer allocateFrame() throws HyracksDataException {
         return joblet.allocateFrame();
     }
-    
+
     @Override
     public void deallocateFrames(int frameCount) {
         joblet.deallocateFrames(frameCount);
@@ -242,7 +247,7 @@
                         final int cIdx = i;
                         executor.execute(new Runnable() {
                             @Override
-							public void run() {
+                            public void run() {
                                 if (aborted) {
                                     return;
                                 }
@@ -252,7 +257,7 @@
                                 thread.setName(displayName + ":" + taskAttemptId + ":" + cIdx);
                                 thread.setPriority(Thread.MIN_PRIORITY);
                                 try {
-                                    pushFrames(collector, writer);
+                                    pushFrames(collector, inputChannelsFromConnectors.get(cIdx), writer);
                                 } catch (HyracksDataException e) {
                                     synchronized (Task.this) {
                                         exceptions.add(e);
@@ -266,7 +271,7 @@
                         });
                     }
                     try {
-                        pushFrames(collectors[0], operator.getInputFrameWriter(0));
+                        pushFrames(collectors[0], inputChannelsFromConnectors.get(0), operator.getInputFrameWriter(0));
                     } finally {
                         sem.acquire(collectors.length - 1);
                     }
@@ -293,15 +298,20 @@
         }
     }
 
-    private void pushFrames(IPartitionCollector collector, IFrameWriter writer) throws HyracksDataException {
+    private void pushFrames(IPartitionCollector collector, List<PartitionChannel> inputChannels, IFrameWriter writer)
+            throws HyracksDataException {
         if (aborted) {
             return;
         }
         try {
             collector.open();
             try {
-                joblet.advertisePartitionRequest(taskAttemptId, collector.getRequiredPartitionIds(), collector,
-                        PartitionState.STARTED);
+                if (inputChannels.size() <= 0) {
+                    joblet.advertisePartitionRequest(taskAttemptId, collector.getRequiredPartitionIds(), collector,
+                            PartitionState.STARTED);
+                } else {
+                    collector.addPartitions(inputChannels);
+                }
                 IFrameReader reader = collector.getReader();
                 reader.open();
                 try {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/profiling/IOCounterLinux.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/profiling/IOCounterLinux.java
index 1e8baa1..804f61d 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/profiling/IOCounterLinux.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/profiling/IOCounterLinux.java
@@ -16,17 +16,18 @@
 package edu.uci.ics.hyracks.control.nc.io.profiling;
 
 import java.io.BufferedReader;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.util.StringTokenizer;
 
 public class IOCounterLinux implements IIOCounter {
     public static final String COMMAND = "iostat";
-    public static final String COMMAND2 = "cat /proc/self/io";
+    public static final String STATFILE = "/proc/self/io";
     public static final int PAGE_SIZE = 4096;
 
-    private final long baseReads;
-    private final long baseWrites;
+    private long baseReads = 0;
+    private long baseWrites = 0;
 
     public IOCounterLinux() {
         baseReads = getReads();
@@ -36,12 +37,12 @@
     @Override
     public long getReads() {
         try {
-            long reads = extractColumn(4);
-            return reads - baseReads;
+            long reads = extractRow(4);
+            return reads;
         } catch (IOException e) {
             try {
-                long reads = extractRow(4);
-                return reads / PAGE_SIZE;
+                long reads = extractColumn(4) * PAGE_SIZE;
+                return reads - baseReads;
             } catch (IOException e2) {
                 return 0;
             }
@@ -51,13 +52,13 @@
     @Override
     public long getWrites() {
         try {
-            long writes = extractColumn(5);
-            return writes - baseWrites;
+            long writes = extractRow(5);
+            long cancelledWrites = extractRow(6);
+            return (writes - cancelledWrites);
         } catch (IOException e) {
             try {
-                long writes = extractRow(5);
-                long cancelledWrites = extractRow(6);
-                return (writes - cancelledWrites) / PAGE_SIZE;
+                long writes = extractColumn(5) * PAGE_SIZE;
+                return writes - baseWrites;
             } catch (IOException e2) {
                 return 0;
             }
@@ -92,7 +93,7 @@
     }
 
     private long extractRow(int rowIndex) throws IOException {
-        BufferedReader reader = exec(COMMAND2);
+        BufferedReader reader = new BufferedReader(new InputStreamReader(new FileInputStream(STATFILE)));
         String line = null;
         long ios = 0;
         int i = 0;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
index 130c967..348a37c5 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
@@ -48,11 +48,14 @@
 
     private final MuxDemux md;
 
+    private final int nBuffers;
+
     private NetworkAddress networkAddress;
 
-    public DatasetNetworkManager(InetAddress inetAddress, IDatasetPartitionManager partitionManager, int nThreads)
-            throws IOException {
+    public DatasetNetworkManager(InetAddress inetAddress, IDatasetPartitionManager partitionManager, int nThreads,
+            int nBuffers) throws IOException {
         this.partitionManager = partitionManager;
+        this.nBuffers = nBuffers;
         md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads,
                 MAX_CONNECTION_ATTEMPTS);
     }
@@ -102,7 +105,7 @@
                 LOGGER.fine("Received initial dataset partition read request for JobId: " + jobId + " partition: "
                         + partition + " on channel: " + ccb);
             }
-            noc = new NetworkOutputChannel(ccb, 1);
+            noc = new NetworkOutputChannel(ccb, nBuffers);
             try {
                 partitionManager.initializeDatasetPartitionReader(jobId, rsId, partition, noc);
             } catch (HyracksException e) {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
index 4d9cd22..8791aa1 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
@@ -47,12 +47,16 @@
 
     private final PartitionManager partitionManager;
 
+    private final int nBuffers;
+
     private final MuxDemux md;
 
     private NetworkAddress networkAddress;
 
-    public NetworkManager(InetAddress inetAddress, PartitionManager partitionManager, int nThreads) throws IOException {
+    public NetworkManager(InetAddress inetAddress, PartitionManager partitionManager, int nThreads, int nBuffers)
+            throws IOException {
         this.partitionManager = partitionManager;
+        this.nBuffers = nBuffers;
         md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads,
                 MAX_CONNECTION_ATTEMPTS);
     }
@@ -99,10 +103,11 @@
             if (LOGGER.isLoggable(Level.FINE)) {
                 LOGGER.fine("Received initial partition request: " + pid + " on channel: " + ccb);
             }
-            noc = new NetworkOutputChannel(ccb, 1);
+            noc = new NetworkOutputChannel(ccb, nBuffers);
             try {
                 partitionManager.registerPartitionRequest(pid, noc);
             } catch (HyracksException e) {
+                e.printStackTrace();
                 noc.abort();
             }
         }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java
index 433c45a..6c4570d 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java
@@ -50,7 +50,9 @@
 
     @Override
     public void deallocate() {
-        partitionFile.delete();
+        if (partitionFile != null) {
+            partitionFile.delete();
+        }
     }
 
     @Override
@@ -59,6 +61,11 @@
             @Override
             public void run() {
                 try {
+                    if (partitionFile == null) {
+                        writer.open();
+                        writer.close();
+                        return;
+                    }
                     IFileHandle fh = ioManager.open(partitionFile, IIOManager.FileReadWriteMode.READ_ONLY,
                             IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
                     try {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java
index 0e9005a..b94d714 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java
@@ -65,15 +65,17 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("open(" + pid + " by " + taId);
         }
-        fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(pid.toString());
-        handle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-        size = 0;
         failed = false;
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (handle == null) {
+            fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(pid.toString());
+            handle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+            size = 0;
+        }
         size += ctx.getIOManager().syncWrite(handle, size, buffer);
     }
 
@@ -87,11 +89,14 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("close(" + pid + " by " + taId);
         }
-        ctx.getIOManager().close(handle);
+        if (handle != null) {
+            ctx.getIOManager().close(handle);
+        }
         if (!failed) {
             manager.registerPartition(pid, taId,
                     new MaterializedPartition(ctx, fRef, executor, (IOManager) ctx.getIOManager()),
-                    PartitionState.COMMITTED);
+                    PartitionState.COMMITTED, taId.getAttempt() == 0 ? false : true);
+
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
index 0e63485..6393979 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializingPipelinedPartition.java
@@ -73,7 +73,9 @@
 
     @Override
     public void deallocate() {
-        fRef.delete();
+        if (fRef != null) {
+            fRef.delete();
+        }
     }
 
     @Override
@@ -82,47 +84,56 @@
             @Override
             public void run() {
                 try {
-                    IFileHandle fh = ioManager.open(fRef, IIOManager.FileReadWriteMode.READ_ONLY,
-                            IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+                    synchronized (MaterializingPipelinedPartition.this) {
+                        while (fRef == null && eos == false) {
+                            MaterializingPipelinedPartition.this.wait();
+                        }
+                    }
+                    IFileHandle fh = fRef == null ? null : ioManager.open(fRef,
+                            IIOManager.FileReadWriteMode.READ_ONLY, IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
                     try {
                         writer.open();
                         try {
-                            long offset = 0;
-                            ByteBuffer buffer = ctx.allocateFrame();
-                            boolean fail = false;
-                            boolean done = false;
-                            while (!fail && !done) {
-                                synchronized (MaterializingPipelinedPartition.this) {
-                                    while (offset >= size && !eos && !failed) {
-                                        try {
-                                            MaterializingPipelinedPartition.this.wait();
-                                        } catch (InterruptedException e) {
-                                            throw new HyracksDataException(e);
+                            if (fh != null) {
+                                long offset = 0;
+                                ByteBuffer buffer = ctx.allocateFrame();
+                                boolean fail = false;
+                                boolean done = false;
+                                while (!fail && !done) {
+                                    synchronized (MaterializingPipelinedPartition.this) {
+                                        while (offset >= size && !eos && !failed) {
+                                            try {
+                                                MaterializingPipelinedPartition.this.wait();
+                                            } catch (InterruptedException e) {
+                                                throw new HyracksDataException(e);
+                                            }
                                         }
+                                        fail = failed;
+                                        done = eos && offset >= size;
                                     }
-                                    fail = failed;
-                                    done = eos && offset >= size;
-                                }
-                                if (fail) {
-                                    writer.fail();
-                                } else if (!done) {
-                                    buffer.clear();
-                                    long readLen = ioManager.syncRead(fh, offset, buffer);
-                                    if (readLen < buffer.capacity()) {
-                                        throw new HyracksDataException("Premature end of file");
+                                    if (fail) {
+                                        writer.fail();
+                                    } else if (!done) {
+                                        buffer.clear();
+                                        long readLen = ioManager.syncRead(fh, offset, buffer);
+                                        if (readLen < buffer.capacity()) {
+                                            throw new HyracksDataException("Premature end of file");
+                                        }
+                                        offset += readLen;
+                                        buffer.flip();
+                                        writer.nextFrame(buffer);
                                     }
-                                    offset += readLen;
-                                    buffer.flip();
-                                    writer.nextFrame(buffer);
                                 }
                             }
                         } finally {
                             writer.close();
                         }
                     } finally {
-                        ioManager.close(fh);
+                        if (fh != null) {
+                            ioManager.close(fh);
+                        }
                     }
-                } catch (HyracksDataException e) {
+                } catch (Exception e) {
                     throw new RuntimeException(e);
                 }
             }
@@ -139,17 +150,23 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("open(" + pid + " by " + taId);
         }
-        fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(pid.toString().replace(":", "$"));
-        handle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
-                IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
         size = 0;
         eos = false;
         failed = false;
-        manager.registerPartition(pid, taId, this, PartitionState.STARTED);
+        manager.registerPartition(pid, taId, this, PartitionState.STARTED, false);
+    }
+
+    private void checkOrCreateFile() throws HyracksDataException {
+        if (fRef == null) {
+            fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(pid.toString().replace(":", "$"));
+            handle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+                    IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+        }
     }
 
     @Override
     public synchronized void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        checkOrCreateFile();
         size += ctx.getIOManager().syncWrite(handle, size, buffer);
         notifyAll();
     }
@@ -165,16 +182,13 @@
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("close(" + pid + " by " + taId);
         }
-        boolean commit = false;
         synchronized (this) {
             eos = true;
-            ctx.getIOManager().close(handle);
+            if (handle != null) {
+                ctx.getIOManager().close(handle);
+            }
             handle = null;
-            commit = !failed;
             notifyAll();
         }
-        if (commit) {
-            manager.updatePartitionState(pid, taId, this, PartitionState.COMMITTED);
-        }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
index ea966c7..b209cc1 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -37,51 +37,68 @@
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 
 public class PartitionManager {
+
     private final NodeControllerService ncs;
 
-    private final Map<PartitionId, List<IPartition>> partitionMap;
+    private final Map<PartitionId, List<IPartition>> availablePartitionMap;
 
     private final DefaultDeallocatableRegistry deallocatableRegistry;
 
     private final IWorkspaceFileFactory fileFactory;
 
+    private final Map<PartitionId, NetworkOutputChannel> partitionRequests = new HashMap<PartitionId, NetworkOutputChannel>();
+
     public PartitionManager(NodeControllerService ncs) {
         this.ncs = ncs;
-        partitionMap = new HashMap<PartitionId, List<IPartition>>();
-        deallocatableRegistry = new DefaultDeallocatableRegistry();
-        fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
+        this.availablePartitionMap = new HashMap<PartitionId, List<IPartition>>();
+        this.deallocatableRegistry = new DefaultDeallocatableRegistry();
+        this.fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext()
+                .getIOManager());
     }
 
-    public void registerPartition(PartitionId pid, TaskAttemptId taId, IPartition partition, PartitionState state)
-            throws HyracksDataException {
-        synchronized (this) {
-            List<IPartition> pList = partitionMap.get(pid);
+    public synchronized void registerPartition(PartitionId pid, TaskAttemptId taId, IPartition partition,
+            PartitionState state, boolean updateToCC) throws HyracksDataException {
+        try {
+            /**
+             * process pending requests
+             */
+            NetworkOutputChannel writer = partitionRequests.remove(pid);
+            if (writer != null) {
+                writer.setFrameSize(partition.getTaskContext().getFrameSize());
+                partition.writeTo(writer);
+                if (!partition.isReusable()) {
+                    return;
+                }
+            }
+
+            /**
+             * put a coming available partition into the available partition map
+             */
+            List<IPartition> pList = availablePartitionMap.get(pid);
             if (pList == null) {
                 pList = new ArrayList<IPartition>();
-                partitionMap.put(pid, pList);
+                availablePartitionMap.put(pid, pList);
             }
             pList.add(partition);
-        }
-        updatePartitionState(pid, taId, partition, state);
-    }
 
-    public void updatePartitionState(PartitionId pid, TaskAttemptId taId, IPartition partition, PartitionState state)
-            throws HyracksDataException {
-        PartitionDescriptor desc = new PartitionDescriptor(pid, ncs.getId(), taId, partition.isReusable());
-        desc.setState(state);
-        try {
-            ncs.getClusterController().registerPartitionProvider(desc);
+            /**
+             * update to CC only when necessary
+             */
+            if (updateToCC) {
+                updatePartitionState(pid, taId, partition, state);
+            }
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
 
     public synchronized IPartition getPartition(PartitionId pid) {
-        return partitionMap.get(pid).get(0);
+        return availablePartitionMap.get(pid).get(0);
     }
 
     public synchronized void unregisterPartitions(JobId jobId, Collection<IPartition> unregisteredPartitions) {
-        for (Iterator<Map.Entry<PartitionId, List<IPartition>>> i = partitionMap.entrySet().iterator(); i.hasNext();) {
+        for (Iterator<Map.Entry<PartitionId, List<IPartition>>> i = availablePartitionMap.entrySet().iterator(); i
+                .hasNext();) {
             Map.Entry<PartitionId, List<IPartition>> e = i.next();
             PartitionId pid = e.getKey();
             if (jobId.equals(pid.getJobId())) {
@@ -95,16 +112,21 @@
 
     public synchronized void registerPartitionRequest(PartitionId partitionId, NetworkOutputChannel writer)
             throws HyracksException {
-        List<IPartition> pList = partitionMap.get(partitionId);
-        if (pList != null && !pList.isEmpty()) {
-            IPartition partition = pList.get(0);
-            writer.setFrameSize(partition.getTaskContext().getFrameSize());
-            partition.writeTo(writer);
-            if (!partition.isReusable()) {
-                partitionMap.remove(partitionId);
+        try {
+            List<IPartition> pList = availablePartitionMap.get(partitionId);
+            if (pList != null && !pList.isEmpty()) {
+                IPartition partition = pList.get(0);
+                writer.setFrameSize(partition.getTaskContext().getFrameSize());
+                partition.writeTo(writer);
+                if (!partition.isReusable()) {
+                    availablePartitionMap.remove(partitionId);
+                }
+            } else {
+                //throw new HyracksException("Request for unknown partition " + partitionId);
+                partitionRequests.put(partitionId, writer);
             }
-        } else {
-            throw new HyracksException("Request for unknown partition " + partitionId);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
         }
     }
 
@@ -115,4 +137,15 @@
     public void close() {
         deallocatableRegistry.close();
     }
+
+    public void updatePartitionState(PartitionId pid, TaskAttemptId taId, IPartition partition, PartitionState state)
+            throws HyracksDataException {
+        PartitionDescriptor desc = new PartitionDescriptor(pid, ncs.getId(), taId, partition.isReusable());
+        desc.setState(state);
+        try {
+            ncs.getClusterController().registerPartitionProvider(desc);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
index 76345bc..c1abdc3 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
@@ -69,7 +69,7 @@
 
     @Override
     public void open() throws HyracksDataException {
-        manager.registerPartition(pid, taId, this, PartitionState.STARTED);
+        manager.registerPartition(pid, taId, this, PartitionState.STARTED, false);
         failed = false;
         pendingConnection = true;
     }
@@ -108,7 +108,6 @@
     public void close() throws HyracksDataException {
         if (!failed) {
             ensureConnected();
-            manager.updatePartitionState(pid, taId, this, PartitionState.COMMITTED);
             delegate.close();
         }
     }
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
index b9ee504..1be5fc6 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
@@ -14,6 +14,9 @@
  */
 package edu.uci.ics.hyracks.control.nc.work;
 
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
@@ -25,6 +28,8 @@
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
 import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
@@ -43,6 +48,7 @@
 import edu.uci.ics.hyracks.api.job.JobFlag;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkInputChannel;
 import edu.uci.ics.hyracks.control.common.deployment.DeploymentUtils;
 import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
 import edu.uci.ics.hyracks.control.common.work.AbstractWork;
@@ -89,8 +95,7 @@
     public void run() {
         try {
             NCApplicationContext appCtx = ncs.getApplicationContext();
-            final Joblet joblet = getOrCreateLocalJoblet(deploymentId, jobId, appCtx, acgBytes == null ? null
-                    : (ActivityClusterGraph) DeploymentUtils.deserialize(acgBytes, deploymentId, appCtx));
+            final Joblet joblet = getOrCreateLocalJoblet(deploymentId, jobId, appCtx, acgBytes);
             final ActivityClusterGraph acg = joblet.getActivityClusterGraph();
 
             IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
@@ -119,12 +124,13 @@
                     LOGGER.info("Initializing " + taId + " -> " + han);
                 }
                 final int partition = tid.getPartition();
-                Task task = new Task(joblet, taId, han.getClass().getName(), ncs.getExecutor(), ncs);
+                List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
+                Task task = new Task(joblet, taId, han.getClass().getName(), ncs.getExecutor(), ncs,
+                        createInputChannels(td, inputs));
                 IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, td.getPartitionCount());
 
                 List<IPartitionCollector> collectors = new ArrayList<IPartitionCollector>();
 
-                List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid);
                 if (inputs != null) {
                     for (int i = 0; i < inputs.size(); ++i) {
                         IConnectorDescriptor conn = inputs.get(i);
@@ -169,13 +175,15 @@
     }
 
     private Joblet getOrCreateLocalJoblet(DeploymentId deploymentId, JobId jobId, INCApplicationContext appCtx,
-            ActivityClusterGraph acg) throws Exception {
+            byte[] acgBytes) throws Exception {
         Map<JobId, Joblet> jobletMap = ncs.getJobletMap();
         Joblet ji = jobletMap.get(jobId);
         if (ji == null) {
-            if (acg == null) {
+            if (acgBytes == null) {
                 throw new NullPointerException("JobActivityGraph was null");
             }
+            ActivityClusterGraph acg = (ActivityClusterGraph) DeploymentUtils.deserialize(acgBytes, deploymentId,
+                    appCtx);
             ji = new Joblet(ncs, deploymentId, jobId, appCtx, acg);
             jobletMap.put(jobId, ji);
         }
@@ -231,4 +239,38 @@
         }
         return factory;
     }
+
+    /**
+     * Create a list of known channels for each input connector
+     * 
+     * @param td
+     *            the task attempt id
+     * @param inputs
+     *            the input connector descriptors
+     * @return a list of known channels, one for each connector
+     * @throws UnknownHostException
+     */
+    private List<List<PartitionChannel>> createInputChannels(TaskAttemptDescriptor td, List<IConnectorDescriptor> inputs)
+            throws UnknownHostException {
+        NetworkAddress[][] inputAddresses = td.getInputPartitionLocations();
+        List<List<PartitionChannel>> channelsForInputConnectors = new ArrayList<List<PartitionChannel>>();
+        if (inputAddresses != null) {
+            for (int i = 0; i < inputAddresses.length; i++) {
+                List<PartitionChannel> channels = new ArrayList<PartitionChannel>();
+                if (inputAddresses[i] != null) {
+                    for (int j = 0; j < inputAddresses[i].length; j++) {
+                        NetworkAddress networkAddress = inputAddresses[i][j];
+                        PartitionId pid = new PartitionId(jobId, inputs.get(i).getConnectorId(), j, td
+                                .getTaskAttemptId().getTaskId().getPartition());
+                        PartitionChannel channel = new PartitionChannel(pid, new NetworkInputChannel(
+                                ncs.getNetworkManager(), new InetSocketAddress(InetAddress.getByAddress(networkAddress
+                                        .getIpAddress()), networkAddress.getPort()), pid, 5));
+                        channels.add(channel);
+                    }
+                }
+                channelsForInputConnectors.add(channels);
+            }
+        }
+        return channelsForInputConnectors;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
index 1b26c00..f894e38 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
@@ -15,11 +15,9 @@
 package edu.uci.ics.hyracks.data.std.util;
 
 import java.io.ByteArrayOutputStream;
-import java.util.logging.Level;
-import java.util.logging.Logger;
+import java.util.Arrays;
 
 public class ByteArrayAccessibleOutputStream extends ByteArrayOutputStream {
-    private static final Logger LOGGER = Logger.getLogger(ByteArrayAccessibleOutputStream.class.getName());
 
     public ByteArrayAccessibleOutputStream() {
         super();
@@ -34,17 +32,45 @@
     }
 
     public void write(int b) {
-        if (LOGGER.isLoggable(Level.FINEST)) {
-            LOGGER.finest("write(byte) value: " + b);
-        }
-        super.write(b);
+        ensureCapacity(count + 1);
+        buf[count] = (byte) b;
+        count += 1;
     }
 
     @Override
-    public void write(byte[] bytes, int offset, int length) {
-        if (LOGGER.isLoggable(Level.FINEST)) {
-            LOGGER.finest("write(byte[], int, int) offset: " + offset + " length" + length);
+    public void write(byte[] b, int off, int len) {
+        if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) - b.length > 0)) {
+            throw new IndexOutOfBoundsException();
         }
-        super.write(bytes, offset, length);
+        ensureCapacity(count + len);
+        System.arraycopy(b, off, buf, count, len);
+        count += len;
+    }
+
+    private void ensureCapacity(int minCapacity) {
+        // overflow-conscious code
+        if (minCapacity - buf.length > 0)
+            grow(minCapacity);
+    }
+
+    /**
+     * Increases the capacity to ensure that it can hold at least the
+     * number of elements specified by the minimum capacity argument.
+     * 
+     * @param minCapacity
+     *            the desired minimum capacity
+     */
+    private void grow(int minCapacity) {
+        // overflow-conscious code
+        int oldCapacity = buf.length;
+        int newCapacity = oldCapacity << 1;
+        if (newCapacity - minCapacity < 0)
+            newCapacity = minCapacity;
+        if (newCapacity < 0) {
+            if (minCapacity < 0) // overflow
+                throw new OutOfMemoryError();
+            newCapacity = Integer.MAX_VALUE;
+        }
+        buf = Arrays.copyOf(buf, newCapacity);
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/ByteBufferInputStream.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/ByteBufferInputStream.java
index 9cdd692..eee4758 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/ByteBufferInputStream.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/ByteBufferInputStream.java
@@ -16,11 +16,8 @@
 
 import java.io.InputStream;
 import java.nio.ByteBuffer;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 public class ByteBufferInputStream extends InputStream {
-    private static final Logger LOGGER = Logger.getLogger(ByteBufferInputStream.class.getName());
 
     private ByteBuffer buffer;
 
@@ -37,20 +34,13 @@
     @Override
     public int read() {
         int remaining = buffer.capacity() - position;
-        int value = remaining > 0 ? (buffer.get(position++) & 0xff) : -1;
-        if (LOGGER.isLoggable(Level.FINEST)) {
-            LOGGER.finest("read(): value: " + value + " remaining: " + remaining + " position: " + position);
-        }
+        int value = remaining > 0 ? (buffer.array()[position++] & 0xff) : -1;
         return value;
     }
 
     @Override
     public int read(byte[] bytes, int offset, int length) {
         int remaining = buffer.capacity() - position;
-        if (LOGGER.isLoggable(Level.FINEST)) {
-            LOGGER.finest("read(bytes[], int, int): remaining: " + remaining + " offset: " + offset + " length: "
-                    + length + " position: " + position);
-        }
         if (remaining == 0) {
             return -1;
         }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index a1d24e7..ea586fc 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -32,6 +32,8 @@
     private final FrameTupleAppender[] appenders;
     private final FrameTupleAccessor tupleAccessor;
     private final ITuplePartitionComputer tpc;
+    private final IHyracksTaskContext ctx;
+    private boolean allocated = false;
 
     public PartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount, IPartitionWriterFactory pwFactory,
             RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
@@ -42,20 +44,22 @@
             try {
                 pWriters[i] = pwFactory.createFrameWriter(i);
                 appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
-                appenders[i].reset(ctx.allocateFrame(), true);
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
         }
         tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
         this.tpc = tpc;
+        this.ctx = ctx;
     }
 
     @Override
     public void close() throws HyracksDataException {
         for (int i = 0; i < pWriters.length; ++i) {
-            if (appenders[i].getTupleCount() > 0) {
-                flushFrame(appenders[i].getBuffer(), pWriters[i]);
+            if (allocated) {
+                if (appenders[i].getTupleCount() > 0) {
+                    flushFrame(appenders[i].getBuffer(), pWriters[i]);
+                }
             }
             pWriters[i].close();
         }
@@ -71,12 +75,15 @@
     public void open() throws HyracksDataException {
         for (int i = 0; i < pWriters.length; ++i) {
             pWriters[i].open();
-            appenders[i].reset(appenders[i].getBuffer(), true);
         }
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (!allocated) {
+            allocateFrames();
+            allocated = true;
+        }
         tupleAccessor.reset(buffer);
         int tupleCount = tupleAccessor.getTupleCount();
         for (int i = 0; i < tupleCount; ++i) {
@@ -87,12 +94,23 @@
                 flushFrame(appenderBuffer, pWriters[h]);
                 appender.reset(appenderBuffer, true);
                 if (!appender.append(tupleAccessor, i)) {
-                    throw new HyracksDataException("Record size (" + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i)) + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
+                    throw new HyracksDataException("Record size ("
+                            + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i))
+                            + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                 }
             }
         }
     }
 
+    /**
+     * @throws HyracksDataException
+     */
+    private void allocateFrames() throws HyracksDataException {
+        for (int i = 0; i < appenders.length; ++i) {
+            appenders[i].reset(ctx.allocateFrame(), true);
+        }
+    }
+
     @Override
     public void fail() throws HyracksDataException {
         for (int i = 0; i < appenders.length; ++i) {
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 ba9ff49..114463f 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
@@ -41,6 +41,7 @@
     private final ByteBuffer outFrame;
     private final FrameTupleAppender appender;
     private final ArrayTupleBuilder tupleBuilder;
+    private boolean outputPartial = false;
 
     private boolean first;
 
@@ -48,6 +49,13 @@
 
     public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
             IAggregatorDescriptor aggregator, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
+            IFrameWriter writer, boolean outputPartial) throws HyracksDataException {
+        this(ctx, groupFields, comparators, aggregator, inRecordDesc, outRecordDesc, writer);
+        this.outputPartial = outputPartial;
+    }
+
+    public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
+            IAggregatorDescriptor aggregator, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
             IFrameWriter writer) throws HyracksDataException {
         this.groupFields = groupFields;
         this.comparators = comparators;
@@ -121,10 +129,13 @@
         for (int j = 0; j < groupFields.length; j++) {
             tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, groupFields[j]);
         }
-        boolean hasOutput = aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor, lastTupleIndex, aggregateState);
+        boolean hasOutput = outputPartial ? aggregator.outputPartialResult(tupleBuilder, lastTupleAccessor,
+                lastTupleIndex, aggregateState) : aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor,
+                lastTupleIndex, aggregateState);
 
-        if (hasOutput && !appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                tupleBuilder.getSize())) {
+        if (hasOutput
+                && !appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                        tupleBuilder.getSize())) {
             FrameUtils.flushFrame(outFrame, writer);
             appender.reset(outFrame, true);
             if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
new file mode 100644
index 0000000..2a28dea
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
@@ -0,0 +1,140 @@
+/*
+ * 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.sort;
+
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+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.IBinaryComparatorFactory;
+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;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+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.preclustered.PreclusteredGroupWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterMergeSort;
+import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterQuickSort;
+import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
+
+/**
+ * Group-by aggregation is pushed before run file generation.
+ * 
+ * @author yingyib
+ */
+public class ExternalSortGroupByRunGenerator implements IFrameWriter {
+    private final IHyracksTaskContext ctx;
+    private final IFrameSorter frameSorter;
+    private final List<IFrameReader> runs;
+    private final int maxSortFrames;
+
+    private final int[] groupFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+    private final RecordDescriptor inRecordDesc;
+    private final RecordDescriptor outRecordDesc;
+
+    public ExternalSortGroupByRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor recordDesc,
+            int framesLimit, int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
+            RecordDescriptor outRecordDesc, Algorithm alg) throws HyracksDataException {
+        this.ctx = ctx;
+        if (alg == Algorithm.MERGE_SORT) {
+            frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    recordDesc);
+        } else {
+            frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    recordDesc);
+        }
+        this.runs = new LinkedList<IFrameReader>();
+        this.maxSortFrames = framesLimit - 1;
+        this.groupFields = groupFields;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregatorFactory = aggregatorFactory;
+        this.inRecordDesc = recordDesc;
+        this.outRecordDesc = outRecordDesc;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        runs.clear();
+        frameSorter.reset();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (frameSorter.getFrameCount() >= maxSortFrames) {
+            flushFramesToRun();
+        }
+        frameSorter.insertFrame(buffer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (frameSorter.getFrameCount() > 0) {
+            if (runs.size() <= 0) {
+                frameSorter.sortFrames();
+            } else {
+                flushFramesToRun();
+            }
+        }
+    }
+
+    private void flushFramesToRun() throws HyracksDataException {
+        frameSorter.sortFrames();
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                ExternalSortGroupByRunGenerator.class.getSimpleName());
+        RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
+
+        //create group-by comparators
+        IBinaryComparator[] comparators = new IBinaryComparator[Math
+                .min(groupFields.length, comparatorFactories.length)];
+        for (int i = 0; i < comparators.length; i++) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc, outRecordDesc,
+                groupFields, groupFields, writer);
+        PreclusteredGroupWriter pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregator,
+                this.inRecordDesc, this.outRecordDesc, writer, true);
+        pgw.open();
+
+        try {
+            frameSorter.flushFrames(pgw);
+        } finally {
+            pgw.close();
+        }
+        frameSorter.reset();
+        runs.add(writer.createReader());
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+    }
+
+    public IFrameSorter getFrameSorter() {
+        return frameSorter;
+    }
+
+    public List<IFrameReader> getRuns() {
+        return runs;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
new file mode 100644
index 0000000..1f9b358
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
@@ -0,0 +1,200 @@
+/*
+ * 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.sort;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+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;
+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.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+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.preclustered.PreclusteredGroupWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+
+/**
+ * Group-by aggregation is pushed into multi-pass merge of external sort.
+ * 
+ * @author yingyib
+ */
+public class ExternalSortGroupByRunMerger {
+
+    private final IHyracksTaskContext ctx;
+    private final List<IFrameReader> runs;
+    private final RecordDescriptor inputRecordDesc;
+    private final RecordDescriptor partialAggRecordDesc;
+    private final RecordDescriptor outRecordDesc;
+    private final int framesLimit;
+    private final IFrameWriter writer;
+    private List<ByteBuffer> inFrames;
+    private ByteBuffer outFrame;
+    private FrameTupleAppender outFrameAppender;
+
+    private IFrameSorter frameSorter; // Used in External sort, no replacement
+                                      // selection
+
+    private final int[] groupFields;
+    private final INormalizedKeyComputer firstKeyNkc;
+    private final IBinaryComparator[] comparators;
+    private final IAggregatorDescriptorFactory mergeAggregatorFactory;
+    private final IAggregatorDescriptorFactory partialAggregatorFactory;
+    private final boolean localSide;
+
+    private final int[] mergeSortFields;
+    private final int[] mergeGroupFields;
+    private IBinaryComparator[] groupByComparators;
+
+    // Constructor for external sort, no replacement selection
+    public ExternalSortGroupByRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+            int[] sortFields, RecordDescriptor inRecordDesc, RecordDescriptor partialAggRecordDesc,
+            RecordDescriptor outRecordDesc, int framesLimit, IFrameWriter writer, int[] groupFields,
+            INormalizedKeyComputer nmk, IBinaryComparator[] comparators,
+            IAggregatorDescriptorFactory partialAggregatorFactory, IAggregatorDescriptorFactory aggregatorFactory,
+            boolean localStage) {
+        this.ctx = ctx;
+        this.frameSorter = frameSorter;
+        this.runs = new LinkedList<IFrameReader>(runs);
+        this.inputRecordDesc = inRecordDesc;
+        this.partialAggRecordDesc = partialAggRecordDesc;
+        this.outRecordDesc = outRecordDesc;
+        this.framesLimit = framesLimit;
+        this.writer = writer;
+
+        this.groupFields = groupFields;
+        this.firstKeyNkc = nmk;
+        this.comparators = comparators;
+        this.mergeAggregatorFactory = aggregatorFactory;
+        this.partialAggregatorFactory = partialAggregatorFactory;
+        this.localSide = localStage;
+
+        //create merge sort fields
+        int numSortFields = sortFields.length;
+        mergeSortFields = new int[numSortFields];
+        for (int i = 0; i < numSortFields; i++) {
+            mergeSortFields[i] = i;
+        }
+
+        //create merge group fields
+        int numGroupFields = groupFields.length;
+        mergeGroupFields = new int[numGroupFields];
+        for (int i = 0; i < numGroupFields; i++) {
+            mergeGroupFields[i] = i;
+        }
+
+        //setup comparators for grouping
+        groupByComparators = new IBinaryComparator[Math.min(mergeGroupFields.length, comparators.length)];
+        for (int i = 0; i < groupByComparators.length; i++) {
+            groupByComparators[i] = comparators[i];
+        }
+    }
+
+    public void process() throws HyracksDataException {
+        IAggregatorDescriptorFactory aggregatorFactory = localSide ? partialAggregatorFactory : mergeAggregatorFactory;
+        IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, partialAggRecordDesc, outRecordDesc,
+                groupFields, groupFields, writer);
+        PreclusteredGroupWriter pgw = new PreclusteredGroupWriter(ctx, groupFields, groupByComparators, aggregator,
+                inputRecordDesc, outRecordDesc, writer, false);
+        try {
+            if (runs.size() <= 0) {
+                pgw.open();
+                if (frameSorter != null && frameSorter.getFrameCount() > 0) {
+                    frameSorter.flushFrames(pgw);
+                }
+                /** recycle sort buffer */
+                frameSorter.close();
+            } else {
+                /** recycle sort buffer */
+                frameSorter.close();
+
+                inFrames = new ArrayList<ByteBuffer>();
+                outFrame = ctx.allocateFrame();
+                outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+                outFrameAppender.reset(outFrame, true);
+                for (int i = 0; i < framesLimit - 1; ++i) {
+                    inFrames.add(ctx.allocateFrame());
+                }
+                int maxMergeWidth = framesLimit - 1;
+                while (runs.size() > maxMergeWidth) {
+                    int generationSeparator = 0;
+                    while (generationSeparator < runs.size() && runs.size() > maxMergeWidth) {
+                        int mergeWidth = Math.min(Math.min(runs.size() - generationSeparator, maxMergeWidth),
+                                runs.size() - maxMergeWidth + 1);
+                        FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortGroupByRunMerger.class
+                                .getSimpleName());
+                        IFrameWriter mergeResultWriter = new RunFileWriter(newRun, ctx.getIOManager());
+
+                        aggregatorFactory = localSide ? mergeAggregatorFactory : partialAggregatorFactory;
+                        aggregator = aggregatorFactory.createAggregator(ctx, partialAggRecordDesc,
+                                partialAggRecordDesc, mergeGroupFields, mergeGroupFields, mergeResultWriter);
+                        pgw = new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators, aggregator,
+                                partialAggRecordDesc, partialAggRecordDesc, mergeResultWriter, true);
+                        pgw.open();
+
+                        IFrameReader[] runCursors = new RunFileReader[mergeWidth];
+                        for (int i = 0; i < mergeWidth; i++) {
+                            runCursors[i] = runs.get(generationSeparator + i);
+                        }
+                        merge(pgw, runCursors);
+                        pgw.close();
+                        runs.subList(generationSeparator, mergeWidth + generationSeparator).clear();
+                        runs.add(generationSeparator++, ((RunFileWriter) mergeResultWriter).createReader());
+                    }
+                }
+                if (!runs.isEmpty()) {
+                    aggregator = mergeAggregatorFactory.createAggregator(ctx, partialAggRecordDesc, outRecordDesc,
+                            mergeGroupFields, mergeGroupFields, writer);
+                    pgw = new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators, aggregator,
+                            partialAggRecordDesc, outRecordDesc, writer, false);
+                    pgw.open();
+                    IFrameReader[] runCursors = new RunFileReader[runs.size()];
+                    for (int i = 0; i < runCursors.length; i++) {
+                        runCursors[i] = runs.get(i);
+                    }
+                    merge(pgw, runCursors);
+                }
+            }
+        } catch (Exception e) {
+            pgw.fail();
+        } finally {
+            pgw.close();
+        }
+    }
+
+    private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
+        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, mergeSortFields,
+                comparators, firstKeyNkc, partialAggRecordDesc);
+        merger.open();
+        try {
+            while (merger.nextFrame(outFrame)) {
+                FrameUtils.flushFrame(outFrame, mergeResultWriter);
+            }
+        } finally {
+            merger.close();
+        }
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
new file mode 100644
index 0000000..cee105b
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
@@ -0,0 +1,274 @@
+/*
+ * 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.sort;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+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;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
+
+/**
+ * This Operator pushes group-by aggregation into the external sort.
+ * After the in-memory sort, it aggregates the sorted data before writing it to a run file.
+ * During the merge phase, it does an aggregation over sorted results.
+ * 
+ * @author yingyib
+ */
+public class SortGroupByOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static final int SORT_ACTIVITY_ID = 0;
+    private static final int MERGE_ACTIVITY_ID = 1;
+
+    private final int framesLimit;
+    private final int[] sortFields;
+    private final int[] groupFields;
+    private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IAggregatorDescriptorFactory mergeAggregatorFactory;
+    private final IAggregatorDescriptorFactory partialAggregatorFactory;
+    private final RecordDescriptor partialAggRecordDesc;
+    private final RecordDescriptor outputRecordDesc;
+    private final boolean finalStage;
+    private Algorithm alg = Algorithm.MERGE_SORT;
+
+    /***
+     * @param spec
+     *            , the Hyracks job specification
+     * @param framesLimit
+     *            , the frame limit for this operator
+     * @param sortFields
+     *            , the fields to sort
+     * @param groupFields
+     *            , the fields to group, which can be a prefix subset of sortFields
+     * @param firstKeyNormalizerFactory
+     *            , the normalized key computer factory of the first key
+     * @param comparatorFactories
+     *            , the comparator factories of sort keys
+     * @param partialAggregatorFactory
+     *            , for aggregating the input of this operator
+     * @param mergeAggregatorFactory
+     *            , for aggregating the intermediate data of this operator
+     * @param partialAggRecordDesc
+     *            , the record descriptor of intermediate data
+     * @param outRecordDesc
+     *            , the record descriptor of output data
+     * @param finalStage
+     *            , whether the operator is used for final stage aggregation
+     */
+    public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory partialAggregatorFactory,
+            IAggregatorDescriptorFactory mergeAggregatorFactory, RecordDescriptor partialAggRecordDesc,
+            RecordDescriptor outRecordDesc, boolean finalStage) {
+        super(spec, 1, 1);
+        this.framesLimit = framesLimit;
+        this.sortFields = sortFields;
+        if (framesLimit <= 1) {
+            throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
+        }
+        this.recordDescriptors[0] = outRecordDesc;
+
+        this.groupFields = groupFields;
+        this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
+        this.comparatorFactories = comparatorFactories;
+        this.mergeAggregatorFactory = mergeAggregatorFactory;
+        this.partialAggregatorFactory = partialAggregatorFactory;
+        this.partialAggRecordDesc = partialAggRecordDesc;
+        this.outputRecordDesc = outRecordDesc;
+        this.finalStage = finalStage;
+    }
+
+    /***
+     * @param spec
+     *            , the Hyracks job specification
+     * @param framesLimit
+     *            , the frame limit for this operator
+     * @param sortFields
+     *            , the fields to sort
+     * @param groupFields
+     *            , the fields to group, which can be a prefix subset of sortFields
+     * @param firstKeyNormalizerFactory
+     *            , the normalized key computer factory of the first key
+     * @param comparatorFactories
+     *            , the comparator factories of sort keys
+     * @param partialAggregatorFactory
+     *            , for aggregating the input of this operator
+     * @param mergeAggregatorFactory
+     *            , for aggregating the intermediate data of this operator
+     * @param partialAggRecordDesc
+     *            , the record descriptor of intermediate data
+     * @param outRecordDesc
+     *            , the record descriptor of output data
+     * @param finalStage
+     *            , whether the operator is used for final stage aggregation
+     * @param alg
+     *            , the in-memory sort algorithm
+     */
+    public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory partialAggregatorFactory,
+            IAggregatorDescriptorFactory mergeAggregatorFactory, RecordDescriptor partialAggRecordDesc,
+            RecordDescriptor outRecordDesc, boolean finalStage, Algorithm alg) {
+        this(spec, framesLimit, sortFields, groupFields, firstKeyNormalizerFactory, comparatorFactories,
+                partialAggregatorFactory, mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, finalStage);
+        this.alg = alg;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
+        MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
+
+        builder.addActivity(this, sa);
+        builder.addSourceEdge(0, sa, 0);
+
+        builder.addActivity(this, ma);
+        builder.addTargetEdge(0, ma, 0);
+
+        builder.addBlockingEdge(sa, ma);
+    }
+
+    public static class SortTaskState extends AbstractStateObject {
+        private List<IFrameReader> runs;
+        private IFrameSorter frameSorter;
+
+        public SortTaskState() {
+        }
+
+        private SortTaskState(JobId jobId, TaskId taskId) {
+            super(jobId, taskId);
+        }
+
+        @Override
+        public void toBytes(DataOutput out) throws IOException {
+
+        }
+
+        @Override
+        public void fromBytes(DataInput in) throws IOException {
+
+        }
+    }
+
+    private class SortActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public SortActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
+                private ExternalSortGroupByRunGenerator runGen;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    runGen = new ExternalSortGroupByRunGenerator(ctx, sortFields,
+                            recordDescProvider.getInputRecordDescriptor(SortActivity.this.getActivityId(), 0),
+                            framesLimit, groupFields, firstKeyNormalizerFactory, comparatorFactories,
+                            partialAggregatorFactory, partialAggRecordDesc, alg);
+                    runGen.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    runGen.nextFrame(buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(
+                            getActivityId(), partition));
+                    runGen.close();
+                    state.runs = runGen.getRuns();
+                    state.frameSorter = runGen.getFrameSorter();
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    runGen.fail();
+                }
+            };
+            return op;
+        }
+    }
+
+    private class MergeActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public MergeActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
+                @Override
+                public void initialize() throws HyracksDataException {
+                    SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
+                            SORT_ACTIVITY_ID), partition));
+                    List<IFrameReader> runs = state.runs;
+                    IFrameSorter frameSorter = state.frameSorter;
+                    int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
+
+                    IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+                    for (int i = 0; i < comparators.length; i++) {
+                        comparators[i] = comparatorFactories[i].createBinaryComparator();
+                    }
+                    INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory
+                            .createNormalizedKeyComputer();
+
+                    ExternalSortGroupByRunMerger merger = new ExternalSortGroupByRunMerger(ctx, frameSorter, runs,
+                            sortFields, recordDescProvider.getInputRecordDescriptor(new ActivityId(odId,
+                                    SORT_ACTIVITY_ID), 0), partialAggRecordDesc, outputRecordDesc, necessaryFrames,
+                            writer, groupFields, nkc, comparators, partialAggregatorFactory, mergeAggregatorFactory,
+                            !finalStage);
+                    merger.process();
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
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 eaf4162..9178094 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
@@ -145,6 +145,7 @@
                             runCursors[i] = runs.get(generationSeparator + i);
                         }
                         merge(mergeResultWriter, runCursors);
+                        mergeResultWriter.close();
                         runs.subList(generationSeparator, mergeWidth + generationSeparator).clear();
                         runs.add(generationSeparator++, ((RunFileWriter) mergeResultWriter).createReader());
                     }
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 00fbe9b..24c8cb9 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
@@ -149,14 +149,10 @@
         return new Comparator<ReferenceEntry>() {
             public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
                 int nmk1 = tp1.getNormalizedKey();
-                int nmk2 = tp1.getNormalizedKey();
-                if (nmk1 > nmk2) {
-                    return 1;
+                int nmk2 = tp2.getNormalizedKey();
+                if (nmk1 != nmk2) {
+                    return ((((long) nmk1) & 0xffffffffL) < (((long) nmk2) & 0xffffffffL)) ? -1 : 1;
                 }
-                if (nmk1 < nmk2) {
-                    return -1;
-                }
-
                 FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
                 FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
                 byte[] b1 = fta1.getBuffer().array();
@@ -171,7 +167,9 @@
                         return c;
                     }
                 }
-                return 0;
+                int runid1 = tp1.getRunid();
+                int runid2 = tp2.getRunid();
+                return runid1 < runid2 ? -1 : (runid1 == runid2 ? 0 : 1);
             }
         };
     }
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 9dd303d..e20211f 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -53,9 +53,9 @@
     public RuntimeContext(INCApplicationContext appCtx) throws HyracksDataException {
         fileMapManager = new TransientFileMapManager();
         ICacheMemoryAllocator allocator = new HeapBufferAllocator();
-        IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
-        bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs,
-                new DelayPageCleanerPolicy(1000), fileMapManager, 32768, 50, 100, threadFactory);
+        IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator, 32768, 50);
+        bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), prs, new DelayPageCleanerPolicy(1000),
+                fileMapManager, 100, threadFactory);
         lcManager = new IndexLifecycleManager();
         ILocalResourceRepositoryFactory localResourceRepositoryFactory = new TransientLocalResourceRepositoryFactory();
         localResourceRepository = localResourceRepositoryFactory.createRepository();
diff --git a/hyracks/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java b/hyracks/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
index e32e409..0f76343 100644
--- a/hyracks/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
+++ b/hyracks/hyracks-ipc/src/main/java/edu/uci/ics/hyracks/ipc/impl/IPCSystem.java
@@ -77,6 +77,7 @@
         Exception exception = null;
         if (message.getFlag() == Message.ERROR) {
             exception = (Exception) message.getPayload();
+            exception.printStackTrace();
         } else {
             payload = message.getPayload();
         }
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
index 2e95cde..6b9364d 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelControlBlock.java
@@ -108,12 +108,20 @@
 
         private ByteBuffer currentReadBuffer;
 
+        private IBufferFactory bufferFactory;
+
         ReadInterface() {
             riEmptyStack = new ArrayDeque<ByteBuffer>();
             credits = 0;
         }
 
         @Override
+        public void setBufferFactory(IBufferFactory bufferFactory, int limit, int frameSize) {
+            this.bufferFactory = bufferFactory;
+            cSet.addPendingCredits(channelId, limit * frameSize);
+        }
+
+        @Override
         public void setFullBufferAcceptor(ICloseableBufferAcceptor fullBufferAcceptor) {
             fba = fullBufferAcceptor;
         }
@@ -130,6 +138,11 @@
                 }
                 if (currentReadBuffer == null) {
                     currentReadBuffer = riEmptyStack.poll();
+                    //if current buffer == null and limit not reached
+                    // factory.createBuffer factory
+                    if (currentReadBuffer == null) {
+                        currentReadBuffer = bufferFactory.createBuffer();
+                    }
                     assert currentReadBuffer != null;
                 }
                 int rSize = Math.min(size, currentReadBuffer.remaining());
@@ -171,6 +184,8 @@
 
         private boolean channelWritabilityState;
 
+        private IBufferFactory bufferFactory;
+
         private final ICloseableBufferAcceptor fba = new ICloseableBufferAcceptor() {
             @Override
             public void accept(ByteBuffer buffer) {
@@ -227,6 +242,22 @@
         }
 
         @Override
+        public void setBufferFactory(IBufferFactory bufferFactory, int limit, int frameSize) {
+            this.bufferFactory = bufferFactory;
+            if (!channelWritabilityState) {
+                cSet.markPendingWrite(channelId);
+            }
+            channelWritabilityState = true;
+            if (eos) {
+                if (LOGGER.isLoggable(Level.WARNING)) {
+                    LOGGER.warning("Received duplicate close() on channel: " + channelId);
+                }
+                return;
+            }
+            eos = true;
+        }
+
+        @Override
         public void setEmptyBufferAcceptor(IBufferAcceptor emptyBufferAcceptor) {
             eba = emptyBufferAcceptor;
         }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IBufferFactory.java
similarity index 69%
copy from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
copy to hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IBufferFactory.java
index fb2d1eb..5abba95 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IBufferFactory.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2013 by The Regents of the University of California
+ * Copyright 2009-2014 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
@@ -12,10 +12,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package edu.uci.ics.hyracks.net.protocols.muxdemux;
 
-package edu.uci.ics.pregelix.dataflow.util;
+import java.nio.ByteBuffer;
 
-public enum StorageType {
-    TreeIndex,
-    LSMIndex
+/**
+ * @author yingyib
+ */
+public interface IBufferFactory {
+
+    public ByteBuffer createBuffer();
+
 }
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
index 2e961fd..eb683eb 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelReadInterface.java
@@ -39,4 +39,17 @@
      * @return the empty buffer acceptor.
      */
     public IBufferAcceptor getEmptyBufferAcceptor();
+
+    /**
+     * Set the buffer factory which is in charge of creating buffers if the request does not
+     * make the number of allocated buffers goes beyond limit
+     * 
+     * @param bufferFactory
+     *            - the buffer factory
+     * @param limit
+     *            - the limit of buffers
+     * @param frameSize
+     *            - the size of each buffer
+     */
+    public void setBufferFactory(IBufferFactory bufferFactory, int limit, int frameSize);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
index e2fb764..42516ea 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/IChannelWriteInterface.java
@@ -39,4 +39,17 @@
      * @return the full buffer acceptor.
      */
     public ICloseableBufferAcceptor getFullBufferAcceptor();
+
+    /**
+     * Set the buffer factory which is in charge of creating buffers if the request does not
+     * make the number of allocated buffers goes beyond limit
+     * 
+     * @param bufferFactory
+     *            - the buffer factory
+     * @param limit
+     *            - the limit of buffers
+     * @param frameSize
+     *            - the size of each buffer
+     */
+    public void setBufferFactory(IBufferFactory bufferFactory, int limit, int frameSize);
 }
\ 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 03d57f5..740c447 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
@@ -16,18 +16,15 @@
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -46,11 +43,9 @@
     private static final int MIN_CLEANED_COUNT_DIFF = 3;
     private static final int PIN_MAX_WAIT_TIME = 50;
 
-    private final int maxOpenFiles;
-
-    private final IIOManager ioManager;
     private final int pageSize;
-    private final int numPages;
+    private final int maxOpenFiles;
+    private final IIOManager ioManager;
     private final CacheBucket[] pageMap;
     private final IPageReplacementStrategy pageReplacementStrategy;
     private final IPageCleanerPolicy pageCleanerPolicy;
@@ -58,27 +53,21 @@
     private final CleanerThread cleanerThread;
     private final Map<Integer, BufferedFileHandle> fileInfoMap;
 
-    private CachedPage[] cachedPages;
+    private List<ICachedPageInternal> cachedPages = new ArrayList<ICachedPageInternal>();
 
     private boolean closed;
 
-    public BufferCache(IIOManager ioManager, ICacheMemoryAllocator allocator,
-            IPageReplacementStrategy pageReplacementStrategy, IPageCleanerPolicy pageCleanerPolicy,
-            IFileMapManager fileMapManager, int pageSize, int numPages, int maxOpenFiles, ThreadFactory threadFactory) {
+    public BufferCache(IIOManager ioManager, IPageReplacementStrategy pageReplacementStrategy,
+            IPageCleanerPolicy pageCleanerPolicy, IFileMapManager fileMapManager, int maxOpenFiles,
+            ThreadFactory threadFactory) {
         this.ioManager = ioManager;
-        this.pageSize = pageSize;
-        this.numPages = numPages;
+        this.pageSize = pageReplacementStrategy.getPageSize();
         this.maxOpenFiles = maxOpenFiles;
         pageReplacementStrategy.setBufferCache(this);
-        pageMap = new CacheBucket[numPages * MAP_FACTOR];
+        pageMap = new CacheBucket[pageReplacementStrategy.getMaxAllowedNumPages() * MAP_FACTOR];
         for (int i = 0; i < pageMap.length; ++i) {
             pageMap[i] = new CacheBucket();
         }
-        ByteBuffer[] buffers = allocator.allocate(pageSize, numPages);
-        cachedPages = new CachedPage[buffers.length];
-        for (int i = 0; i < buffers.length; ++i) {
-            cachedPages[i] = new CachedPage(i, buffers[i], pageReplacementStrategy);
-        }
         this.pageReplacementStrategy = pageReplacementStrategy;
         this.pageCleanerPolicy = pageCleanerPolicy;
         this.fileMapManager = fileMapManager;
@@ -96,7 +85,7 @@
 
     @Override
     public int getNumPages() {
-        return numPages;
+        return pageReplacementStrategy.getMaxAllowedNumPages();
     }
 
     private void pinSanityCheck(long dpid) throws HyracksDataException {
@@ -338,7 +327,8 @@
         StringBuilder buffer = new StringBuilder();
         buffer.append("Buffer cache state\n");
         buffer.append("Page Size: ").append(pageSize).append('\n');
-        buffer.append("Number of physical pages: ").append(numPages).append('\n');
+        buffer.append("Number of physical pages: ").append(pageReplacementStrategy.getMaxAllowedNumPages())
+                .append('\n');
         buffer.append("Hash table size: ").append(pageMap.length).append('\n');
         buffer.append("Page Map:\n");
         int nCachedPages = 0;
@@ -416,88 +406,9 @@
         }
     }
 
-    private class CachedPage implements ICachedPageInternal {
-        private final int cpid;
-        private final ByteBuffer buffer;
-        private final AtomicInteger pinCount;
-        private final AtomicBoolean dirty;
-        private final ReadWriteLock latch;
-        private final Object replacementStrategyObject;
-        volatile long dpid;
-        CachedPage next;
-        volatile boolean valid;
-
-        public CachedPage(int cpid, ByteBuffer buffer, IPageReplacementStrategy pageReplacementStrategy) {
-            this.cpid = cpid;
-            this.buffer = buffer;
-            pinCount = new AtomicInteger();
-            dirty = new AtomicBoolean();
-            latch = new ReentrantReadWriteLock(true);
-            replacementStrategyObject = pageReplacementStrategy.createPerPageStrategyObject(cpid);
-            dpid = -1;
-            valid = false;
-        }
-
-        public void reset(long dpid) {
-            this.dpid = dpid;
-            dirty.set(false);
-            valid = false;
-            pageReplacementStrategy.notifyCachePageReset(this);
-        }
-
-        public void invalidate() {
-            reset(-1);
-        }
-
-        @Override
-        public ByteBuffer getBuffer() {
-            return buffer;
-        }
-
-        @Override
-        public Object getReplacementStrategyObject() {
-            return replacementStrategyObject;
-        }
-
-        @Override
-        public boolean pinIfGoodVictim() {
-            return pinCount.compareAndSet(0, 1);
-        }
-
-        @Override
-        public int getCachedPageId() {
-            return cpid;
-        }
-
-        @Override
-        public void acquireReadLatch() {
-            latch.readLock().lock();
-        }
-
-        @Override
-        public void acquireWriteLatch() {
-            latch.writeLock().lock();
-        }
-
-        @Override
-        public void releaseReadLatch() {
-            latch.readLock().unlock();
-        }
-
-        @Override
-        public void releaseWriteLatch(boolean markDirty) {
-            if (markDirty) {
-                if (dirty.compareAndSet(false, true)) {
-                    pinCount.incrementAndGet();
-                }
-            }
-            latch.writeLock().unlock();
-        }
-    }
-
     @Override
     public ICachedPageInternal getPage(int cpid) {
-        return cachedPages[cpid];
+        return cachedPages.get(cpid);
     }
 
     private class CleanerThread extends Thread {
@@ -564,8 +475,9 @@
             try {
                 while (true) {
                     pageCleanerPolicy.notifyCleanCycleStart(this);
+                    int numPages = pageReplacementStrategy.getNumPages();
                     for (int i = 0; i < numPages; ++i) {
-                        CachedPage cPage = cachedPages[i];
+                        CachedPage cPage = (CachedPage) cachedPages.get(i);
                         cleanPage(cPage, false);
                     }
                     if (shutdownStart) {
@@ -715,7 +627,7 @@
             } else {
                 pinCount = cPage.pinCount.get();
             }
-            if (pinCount != 0) {
+            if (pinCount > 0) {
                 throw new IllegalStateException("Page is pinned and file is being closed. Pincount is: " + pinCount);
             }
             cPage.invalidate();
@@ -808,7 +720,11 @@
     }
 
     @Override
+    public void addPage(ICachedPageInternal page) {
+        cachedPages.add(page);
+    }
+
     public void dumpState(OutputStream os) throws IOException {
         os.write(dumpState().getBytes());
     }
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/CachedPage.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/CachedPage.java
new file mode 100644
index 0000000..d57a356
--- /dev/null
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/CachedPage.java
@@ -0,0 +1,105 @@
+/*
+ * 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.common.buffercache;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * @author yingyib
+ */
+class CachedPage implements ICachedPageInternal {
+    final int cpid;
+    final ByteBuffer buffer;
+    final AtomicInteger pinCount;
+    final AtomicBoolean dirty;
+    final ReadWriteLock latch;
+    private final Object replacementStrategyObject;
+    private final IPageReplacementStrategy pageReplacementStrategy;
+    volatile long dpid;
+    CachedPage next;
+    volatile boolean valid;
+
+    public CachedPage(int cpid, ByteBuffer buffer, IPageReplacementStrategy pageReplacementStrategy) {
+        this.cpid = cpid;
+        this.buffer = buffer;
+        this.pageReplacementStrategy = pageReplacementStrategy;
+        pinCount = new AtomicInteger();
+        dirty = new AtomicBoolean();
+        latch = new ReentrantReadWriteLock(true);
+        replacementStrategyObject = pageReplacementStrategy.createPerPageStrategyObject(cpid);
+        dpid = -1;
+        valid = false;
+    }
+
+    public void reset(long dpid) {
+        this.dpid = dpid;
+        dirty.set(false);
+        valid = false;
+        pageReplacementStrategy.notifyCachePageReset(this);
+    }
+
+    public void invalidate() {
+        reset(-1);
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public Object getReplacementStrategyObject() {
+        return replacementStrategyObject;
+    }
+
+    @Override
+    public boolean pinIfGoodVictim() {
+        return pinCount.compareAndSet(0, 1);
+    }
+
+    @Override
+    public int getCachedPageId() {
+        return cpid;
+    }
+
+    @Override
+    public void acquireReadLatch() {
+        latch.readLock().lock();
+    }
+
+    @Override
+    public void acquireWriteLatch() {
+        latch.writeLock().lock();
+    }
+
+    @Override
+    public void releaseReadLatch() {
+        latch.readLock().unlock();
+    }
+
+    @Override
+    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/ClockPageReplacementStrategy.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
index ec97344..611bf48 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
@@ -24,9 +24,16 @@
     private final Lock lock;
     private IBufferCacheInternal bufferCache;
     private int clockPtr;
+    private ICacheMemoryAllocator allocator;
+    private int numPages = 0;
+    private final int pageSize;
+    private final int maxAllowedNumPages;
 
-    public ClockPageReplacementStrategy() {
+    public ClockPageReplacementStrategy(ICacheMemoryAllocator allocator, int pageSize, int maxAllowedNumPages) {
         this.lock = new ReentrantLock();
+        this.allocator = allocator;
+        this.pageSize = pageSize;
+        this.maxAllowedNumPages = maxAllowedNumPages;
         clockPtr = 0;
     }
 
@@ -53,38 +60,83 @@
     @Override
     public ICachedPageInternal findVictim() {
         lock.lock();
+        ICachedPageInternal cachedPage = null;
         try {
-            int startClockPtr = clockPtr;
-            int cycleCount = 0;
-            do {
-                ICachedPageInternal cPage = bufferCache.getPage(clockPtr);
-
-                /*
-                 * We do two things here:
-                 * 1. If the page has been accessed, then we skip it -- The CAS would return
-                 * false if the current value is false which makes the page a possible candidate
-                 * for replacement.
-                 * 2. We check with the buffer manager if it feels its a good idea to use this
-                 * page as a victim.
-                 */
-                AtomicBoolean accessedFlag = getPerPageObject(cPage);
-                if (!accessedFlag.compareAndSet(true, false)) {
-                    if (cPage.pinIfGoodVictim()) {
-                        return cPage;
-                    }
-                }
-                clockPtr = (clockPtr + 1) % bufferCache.getNumPages();
-                if (clockPtr == startClockPtr) {
-                    ++cycleCount;
-                }
-            } while (cycleCount < MAX_UNSUCCESSFUL_CYCLE_COUNT);
+            if (numPages >= maxAllowedNumPages) {
+                cachedPage = findVictimByEviction();
+            } else {
+                cachedPage = allocatePage();
+            }
         } finally {
             lock.unlock();
         }
+        return cachedPage;
+    }
+
+    private ICachedPageInternal findVictimByEviction() {
+        int startClockPtr = clockPtr;
+        int cycleCount = 0;
+        do {
+            ICachedPageInternal cPage = bufferCache.getPage(clockPtr);
+
+            /*
+             * We do two things here:
+             * 1. If the page has been accessed, then we skip it -- The CAS would return
+             * false if the current value is false which makes the page a possible candidate
+             * for replacement.
+             * 2. We check with the buffer manager if it feels its a good idea to use this
+             * page as a victim.
+             */
+            AtomicBoolean accessedFlag = getPerPageObject(cPage);
+            if (!accessedFlag.compareAndSet(true, false)) {
+                if (cPage.pinIfGoodVictim()) {
+                    return cPage;
+                }
+            }
+            clockPtr = (clockPtr + 1) % numPages;
+            if (clockPtr == startClockPtr) {
+                ++cycleCount;
+            }
+        } while (cycleCount < MAX_UNSUCCESSFUL_CYCLE_COUNT);
+        return null;
+    }
+
+    @Override
+    public int getNumPages() {
+        int retNumPages = 0;
+        lock.lock();
+        try {
+            retNumPages = numPages;
+        } finally {
+            lock.unlock();
+        }
+        return retNumPages;
+    }
+
+    private ICachedPageInternal allocatePage() {
+        CachedPage cPage = new CachedPage(numPages, allocator.allocate(pageSize, 1)[0], this);
+        bufferCache.addPage(cPage);
+        numPages++;
+        AtomicBoolean accessedFlag = getPerPageObject(cPage);
+        if (!accessedFlag.compareAndSet(true, false)) {
+            if (cPage.pinIfGoodVictim()) {
+                return cPage;
+            }
+        }
         return null;
     }
 
     private AtomicBoolean getPerPageObject(ICachedPageInternal cPage) {
         return (AtomicBoolean) cPage.getReplacementStrategyObject();
     }
+
+    @Override
+    public int getPageSize() {
+        return pageSize;
+    }
+
+    @Override
+    public int getMaxAllowedNumPages() {
+        return maxAllowedNumPages;
+    }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCacheInternal.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCacheInternal.java
index 4c9e949..cd2e853 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCacheInternal.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IBufferCacheInternal.java
@@ -16,4 +16,6 @@
 
 public interface IBufferCacheInternal extends IBufferCache {
     public ICachedPageInternal getPage(int cpid);
+
+    public void addPage(ICachedPageInternal page);
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IPageReplacementStrategy.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IPageReplacementStrategy.java
index b6bfdc3..0adcf68 100644
--- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IPageReplacementStrategy.java
+++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/IPageReplacementStrategy.java
@@ -24,4 +24,10 @@
     public void notifyCachePageAccess(ICachedPageInternal cPage);
 
     public ICachedPageInternal findVictim();
+
+    public int getNumPages();
+
+    public int getPageSize();
+
+    public int getMaxAllowedNumPages();
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index dd85945..2370b84 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -79,10 +79,10 @@
     public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
         if (bufferCache == null) {
             ICacheMemoryAllocator allocator = new HeapBufferAllocator();
-            IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
+            IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator, pageSize, numPages);
             IFileMapProvider fileMapProvider = getFileMapProvider(ctx);
-            bufferCache = new BufferCache(ctx.getIOManager(), allocator, prs, new DelayPageCleanerPolicy(1000),
-                    (IFileMapManager) fileMapProvider, pageSize, numPages, maxOpenFiles, threadFactory);
+            bufferCache = new BufferCache(ctx.getIOManager(), prs, new DelayPageCleanerPolicy(1000),
+                    (IFileMapManager) fileMapProvider, maxOpenFiles, threadFactory);
         }
         return bufferCache;
     }
diff --git a/pom.xml b/pom.xml
index 654afc8..330d2bb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,7 +25,7 @@
     <jvm.extraargs />
 
     <!-- Definition of tests in various categories which may be excluded -->
-    <hanging.pregelix.tests>**/pregelix/**/*.java</hanging.pregelix.tests>
+    <hanging.pregelix.tests>**/pregelix/**/FailureRecovery*.java</hanging.pregelix.tests>
     <hivesterix.perf.tests>**/hivesterix/perf/PerfTestSuite.java</hivesterix.perf.tests>
     <global.test.includes>**/*TestSuite.java,**/*Test.java</global.test.includes>
     <global.test.excludes>**/Abstract*.java,${hanging.pregelix.tests},${hivesterix.perf.tests}</global.test.excludes>
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MessageCombiner.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MessageCombiner.java
index fa03c0c..81ac0d0 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MessageCombiner.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/MessageCombiner.java
@@ -60,6 +60,17 @@
     public abstract void stepPartial(I vertexIndex, M msg) throws HyracksDataException;
 
     /**
+     * step call for partial combiner
+     * 
+     * @param vertexIndex
+     *            the receiver vertex identifier
+     * @param partialAggregate
+     *            a partial aggregate value
+     * @throws HyracksDataException
+     */
+    public abstract void stepPartial2(I vertexIndex, P partialAggregate) throws HyracksDataException;
+
+    /**
      * step call for global combiner
      * 
      * @param vertexIndex
@@ -71,7 +82,14 @@
     public abstract void stepFinal(I vertexIndex, P partialAggregate) throws HyracksDataException;
 
     /**
-     * finish partial combiner
+     * finish partial combiner at the second aggregate stage (if any)
+     * 
+     * @return the intermediate combined message of type P
+     */
+    public abstract P finishPartial2();
+
+    /**
+     * finish partial combiner at the first aggregate stage
      * 
      * @return the intermediate combined message of type P
      */
@@ -112,7 +130,25 @@
     /**
      * @return the accumulated byte size
      */
+    public int estimateAccumulatedStateByteSizePartial2(I vertexIndex, P partialAggregate) throws HyracksDataException {
+        return 0;
+    }
+
+    /**
+     * @return the accumulated byte size
+     */
     public int estimateAccumulatedStateByteSizeFinal(I vertexIndex, P partialAggregate) throws HyracksDataException {
         return 0;
     }
+
+    /**
+     * set the intermediate combine result
+     * 
+     * @param p
+     *            the intermediate combine result
+     */
+    public void setPartialCombineState(P p) {
+        throw new IllegalStateException("customized message combiner implementation does not implement this method!");
+    }
+
 }
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 8135479..a93d744 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
@@ -53,11 +53,10 @@
 @SuppressWarnings("rawtypes")
 public abstract class Vertex<I extends WritableComparable, V extends Writable, E extends Writable, M extends WritableSizable>
         implements Writable {
-    private static long superstep = 0;
-    /** Class-wide number of vertices */
-    private static long numVertices = -1;
-    /** Class-wide number of edges */
-    private static long numEdges = -1;
+    /** task context, only used in scanners */
+    public static TaskAttemptContext taskContext;
+    /** vertex context */
+    private VertexContext context;
     /** Vertex id */
     private I vertexId = null;
     /** Vertex value */
@@ -68,8 +67,6 @@
     boolean halt = false;
     /** List of incoming messages from the previous superstep */
     private final List<M> msgList = new ArrayList<M>();
-    /** map context */
-    private static TaskAttemptContext context = null;
     /** a delegate for hyracks stuff */
     private VertexDelegate<I, V, E, M> delegate = new VertexDelegate<I, V, E, M>(this);
     /** this vertex is updated or not */
@@ -234,19 +231,19 @@
     /**
      * Vote to halt. Once all vertex vote to halt and no more messages, a
      * Pregelix job will terminate.
-     * 
      * The state of the current vertex value is saved.
      */
     public final void voteToHalt() {
         halt = true;
         updated = true;
     }
-    
+
     /**
      * Vote to halt. Once all vertex vote to halt and no more messages, a
      * Pregelix job will terminate.
      * 
-     * @param update whether or not to save the vertex value
+     * @param update
+     *            whether or not to save the vertex value
      */
     public final void voteToHalt(boolean update) {
         halt = true;
@@ -255,18 +252,18 @@
 
     /**
      * Activate a halted vertex such that it is alive again.
-     * 
      * The state of the current vertex value is saved.
      */
     public final void activate() {
         halt = false;
         updated = true;
     }
-    
+
     /**
      * Activate a halted vertex such that it is alive again.
      * 
-     * @param update whether or not to save the vertex value
+     * @param update
+     *            whether or not to save the vertex value
      */
     public final void activate(boolean update) {
         halt = false;
@@ -473,16 +470,6 @@
     }
 
     /**
-     * Set the global superstep for all the vertices (internal use)
-     * 
-     * @param superstep
-     *            New superstep
-     */
-    public static final void setSuperstep(long superstep) {
-        Vertex.superstep = superstep;
-    }
-
-    /**
      * Add an outgoing edge into the vertex
      * 
      * @param edge
@@ -553,18 +540,8 @@
      * 
      * @return the current superstep number
      */
-    public static final long getSuperstep() {
-        return superstep;
-    }
-
-    /**
-     * Set the total number of vertices from the last superstep.
-     * 
-     * @param numVertices
-     *            Aggregate vertices in the last superstep
-     */
-    public static final void setNumVertices(long numVertices) {
-        Vertex.numVertices = numVertices;
+    public final long getSuperstep() {
+        return context.getSuperstep();
     }
 
     /**
@@ -572,18 +549,8 @@
      * 
      * @return the number of vertexes in the graph
      */
-    public static final long getNumVertices() {
-        return numVertices;
-    }
-
-    /**
-     * Set the total number of edges from the last superstep.
-     * 
-     * @param numEdges
-     *            Aggregate edges in the last superstep
-     */
-    public static void setNumEdges(long numEdges) {
-        Vertex.numEdges = numEdges;
+    public final long getNumVertices() {
+        return context.getNumVertices();
     }
 
     /**
@@ -591,15 +558,19 @@
      * 
      * @return the number of edges in the graph
      */
-    public static final long getNumEdges() {
-        return numEdges;
+    public final long getNumEdges() {
+        return context.getNumVertices();
     }
 
     /**
      * Pregelix internal use only
      */
-    public static final TaskAttemptContext getContext() {
-        return context;
+    public final TaskAttemptContext getContext() {
+        if (context != null) {
+            return context.getContext();
+        } else {
+            return taskContext;
+        }
     }
 
     @Override
@@ -614,6 +585,26 @@
     }
 
     /**
+     * called *once* per partition at the start of each iteration,
+     * before calls to open() or compute()
+     * Users can override this method to configure the pregelix job
+     * and vertex state.
+     */
+    public void configure(Configuration conf) {
+
+    }
+    
+    /**
+     * called *once* per partition at the end of each iteration,
+     * before calls to compute() or close()
+     * Users can override this method to configure the pregelix job
+     * and vertex state.
+     */
+    public void endSuperstep(Configuration conf) {
+
+    }
+
+    /**
      * called immediately before invocations of compute() on a vertex
      * Users can override this method to initiate the state for a vertex
      * before the compute() invocations
@@ -659,4 +650,22 @@
         return terminatePartition;
     }
 
+    /**
+     * Set the vertex context
+     * 
+     * @param ctx
+     */
+    public void setVertexContext(VertexContext ctx) {
+        this.context = ctx;
+    }
+
+    /***
+     * Get the vertex context
+     * 
+     * @return the vertex context
+     */
+    public VertexContext getVertexContext() {
+        return this.context;
+    }
+
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexContext.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexContext.java
new file mode 100644
index 0000000..b98fec0
--- /dev/null
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexContext.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.api.graph;
+
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * The vertex context contains global states like superstep, the number of vertices, the number of edges
+ */
+public class VertexContext {
+
+    private long superstep = 1;
+    private long numVertices = 0;
+    private long numEdges = 0;
+    private TaskAttemptContext context;
+
+    public VertexContext() {
+    }
+
+    public long getSuperstep() {
+        return superstep;
+    }
+
+    public long getNumVertices() {
+        return numVertices;
+    }
+
+    public long getNumEdges() {
+        return numEdges;
+    }
+
+    public TaskAttemptContext getContext() {
+        if (context == null) {
+            throw new IllegalStateException("Job context has not been set.");
+        }
+        return context;
+    }
+
+    public void setSuperstep(long superstep) {
+        this.superstep = superstep;
+    }
+
+    public void setContext(TaskAttemptContext context) {
+        if (context == null) {
+            throw new IllegalStateException("Do not set null job context.");
+        }
+        this.context = context;
+    }
+
+    public void setNumEdges(long numEdges) {
+        this.numEdges = numEdges;
+    }
+
+    public void setNumVertices(long numVertices) {
+        this.numVertices = numVertices;
+    }
+
+}
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
index a05d168..846e6b5 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/VertexDelegate.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.pregelix.api.io.Pointable;
 import edu.uci.ics.pregelix.api.util.FrameTupleUtils;
 
 @SuppressWarnings("rawtypes")
@@ -59,6 +60,8 @@
     /** whether alive message should be pushed out */
     private boolean pushAlive;
 
+    private boolean pointableMsg = false;
+
     public VertexDelegate(Vertex vertex) {
         this.vertex = vertex;
     }
@@ -85,13 +88,23 @@
          * send out message along message channel
          */
         try {
-            message.reset();
-            DataOutput outputMsg = message.getDataOutput();
-            id.write(outputMsg);
-            message.addFieldEndOffset();
-            msg.write(outputMsg);
-            message.addFieldEndOffset();
-            FrameTupleUtils.flushTuple(appenderMsg, message, msgWriter);
+            if (pointableMsg) {
+                FrameTupleUtils.flushPointableKeyValueTuple(appenderMsg, msgWriter, (Pointable) id, (Pointable) msg);
+            } else {
+                if ((id instanceof Pointable) && (msg instanceof Pointable)) {
+                    FrameTupleUtils
+                            .flushPointableKeyValueTuple(appenderMsg, msgWriter, (Pointable) id, (Pointable) msg);
+                    pointableMsg = true;
+                } else {
+                    message.reset();
+                    DataOutput outputMsg = message.getDataOutput();
+                    id.write(outputMsg);
+                    message.addFieldEndOffset();
+                    msg.write(outputMsg);
+                    message.addFieldEndOffset();
+                    FrameTupleUtils.flushTuple(appenderMsg, message, msgWriter);
+                }
+            }
         } catch (Exception e) {
             throw new IllegalStateException(e);
         }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/Pointable.java
similarity index 74%
copy from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
copy to pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/Pointable.java
index fb2d1eb..4217098 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/io/Pointable.java
@@ -13,9 +13,16 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.api.io;
 
-public enum StorageType {
-    TreeIndex,
-    LSMIndex
+public interface Pointable {
+
+    public byte[] getByteArray();
+
+    public int getStartOffset();
+
+    public int getLength();
+    
+    public int set(byte[] data, int offset);
+
 }
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 f2c9c84..fb04b01 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
@@ -27,9 +27,10 @@
 import edu.uci.ics.pregelix.api.graph.VertexPartitioner;
 import edu.uci.ics.pregelix.api.io.VertexInputFormat;
 import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
-import edu.uci.ics.pregelix.api.util.HadoopCountersGlobalAggregateHook;
-import edu.uci.ics.pregelix.api.util.GlobalCountAggregator;
+import edu.uci.ics.pregelix.api.util.GlobalEdgeCountAggregator;
+import edu.uci.ics.pregelix.api.util.GlobalVertexCountAggregator;
 import edu.uci.ics.pregelix.api.util.HadoopCountersAggregator;
+import edu.uci.ics.pregelix.api.util.HadoopCountersGlobalAggregateHook;
 
 /**
  * This class represents a Pregelix job.
@@ -92,9 +93,24 @@
     /** period */
     public static final String PERIOD_STR = ".";
     /** the names of the aggregator classes active for all vertex types */
-    public static final String[] DEFAULT_GLOBAL_AGGREGATOR_CLASSES = { GlobalCountAggregator.class.getName() };
+    public static final String[] DEFAULT_GLOBAL_AGGREGATOR_CLASSES = { GlobalVertexCountAggregator.class.getName(),
+            GlobalEdgeCountAggregator.class.getName() };
     /** The name of an optional class that aggregates all Vertexes into mapreduce.Counters */
     public static final String COUNTERS_AGGREGATOR_CLASS = "pregelix.aggregatedCountersClass";
+    /** the group-by algorithm */
+    public static final String GROUPING_ALGORITHM = "pregelix.groupalg";
+    /** the memory assigned to group-by */
+    public static final String GROUPING_MEM = "pregelix.groupmem";
+    /** the memory assigned for the sort operator */
+    public static final String SORT_MEM = "pregelix.sortmem";
+    /** the number of workers */
+    public static final String NUM_WORKERS = "pregelix.numworkers";
+    /** the application allows to skip combiner key during aggregations */
+    public static final String SKIP_COMBINER_KEY = "pregelix.skipCombinerKey";
+    /** the merge connector */
+    public static final String MERGE_CONNECTOR = "pregelix.merge";
+    /** the maximum allowed iteration */
+    public static final String MAX_ITERATION="pregelix.maxiteration";
 
     /**
      * Construct a Pregelix job from an existing configuration
@@ -290,15 +306,87 @@
         getConfiguration().setBoolean(DYNAMIC_OPTIMIZATION, dynamicOpt);
     }
 
+    /**
+     * Set the counter aggregator class
+     * 
+     * @param aggClass
+     */
     final public void setCounterAggregatorClass(Class<? extends HadoopCountersAggregator<?, ?, ?, ?, ?>> aggClass) {
         if (Modifier.isAbstract(aggClass.getModifiers())) {
-            throw new IllegalArgumentException("Aggregate class must be a concrete class, not an abstract one! (was " + aggClass.getName() + ")");
+            throw new IllegalArgumentException("Aggregate class must be a concrete class, not an abstract one! (was "
+                    + aggClass.getName() + ")");
         }
         getConfiguration().setClass(COUNTERS_AGGREGATOR_CLASS, aggClass, HadoopCountersAggregator.class);
         addGlobalAggregatorClass(aggClass);
         setIterationCompleteReporterHook(HadoopCountersGlobalAggregateHook.class);
     }
 
+    /**
+     * Set the group-by algorithm: sort-true or hash-false
+     * 
+     * @param sortOrHash
+     */
+    final public void setGroupByAlgorithm(boolean sortOrHash) {
+        getConfiguration().setBoolean(GROUPING_ALGORITHM, sortOrHash);
+    }
+
+    /**
+     * Set the memory buget for group-by operators (only hash-based)
+     * 
+     * @param numberOfPages
+     */
+    final public void setGroupByMemoryLimit(int numberOfPages) {
+        getConfiguration().setInt(GROUPING_MEM, numberOfPages);
+    }
+
+    /**
+     * Set the memory buget for sort operators (only hash-based)
+     * 
+     * @param numberOfPages
+     */
+    final public void setSortMemoryLimit(int numberOfPages) {
+        getConfiguration().setInt(SORT_MEM, numberOfPages);
+    }
+
+    /**
+     * Set the number of workers
+     * 
+     * @param numWorkers
+     */
+    final public void setNumWorkers(int numWorkers) {
+        getConfiguration().setInt(NUM_WORKERS, numWorkers);
+    }
+
+    /**
+     * Whether an application allows to skip the combiner key during message combination,
+     * this is a performance improvement tip.
+     * By default, the key is not skipped
+     * 
+     * @param skip
+     *            true to skip; otherwise, not.
+     */
+    final public void setSkipCombinerKey(boolean skip) {
+        getConfiguration().setBoolean(SKIP_COMBINER_KEY, skip);
+    }
+    
+    /**
+     * Whether to use merge connector
+     * 
+     * @param merge
+     */
+    final public void setMergeConnector(boolean merge){
+        getConfiguration().setBoolean(MERGE_CONNECTOR, merge);
+    }
+    
+    /***
+     * Set the maximum allowed iteration
+     * 
+     * @param iteration
+     */
+    final public void setMaxIteration(int iteration){
+        getConfiguration().setInt(MAX_ITERATION, iteration);
+    }
+
     @Override
     public String toString() {
         return getJobName();
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 bef9aa9..1198a3e 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
@@ -29,7 +29,6 @@
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.util.ReflectionUtils;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
@@ -50,7 +49,7 @@
  * them.
  */
 public class BspUtils {
-    
+
     public static final String TMP_DIR = "/tmp/";
     private static final String COUNTERS_VALUE_ON_ITERATION = ".counters.valueOnIter.";
     private static final String COUNTERS_LAST_ITERATION_COMPLETED = ".counters.lastIterCompleted";
@@ -80,8 +79,12 @@
     public static <I extends WritableComparable, V extends Writable, E extends Writable, M extends WritableSizable> VertexInputFormat<I, V, E, M> createVertexInputFormat(
             Configuration conf) {
         Class<? extends VertexInputFormat<I, V, E, M>> vertexInputFormatClass = getVertexInputFormatClass(conf);
-        VertexInputFormat<I, V, E, M> inputFormat = ReflectionUtils.newInstance(vertexInputFormatClass, conf);
-        return inputFormat;
+        try {
+            VertexInputFormat<I, V, E, M> inputFormat = vertexInputFormatClass.newInstance();
+            return inputFormat;
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
     }
 
     /**
@@ -109,7 +112,11 @@
     public static <I extends WritableComparable, V extends Writable, E extends Writable> VertexOutputFormat<I, V, E> createVertexOutputFormat(
             Configuration conf) {
         Class<? extends VertexOutputFormat<I, V, E>> vertexOutputFormatClass = getVertexOutputFormatClass(conf);
-        return ReflectionUtils.newInstance(vertexOutputFormatClass, conf);
+        try {
+            return vertexOutputFormatClass.newInstance();
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
     }
 
     /**
@@ -172,7 +179,11 @@
     public static <I extends WritableComparable, M extends WritableSizable, P extends Writable> MessageCombiner<I, M, P> createMessageCombiner(
             Configuration conf) {
         Class<? extends MessageCombiner<I, M, P>> vertexCombinerClass = getMessageCombinerClass(conf);
-        return ReflectionUtils.newInstance(vertexCombinerClass, conf);
+        try {
+            return vertexCombinerClass.newInstance();
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
     }
 
     /**
@@ -184,7 +195,11 @@
      */
     public static NormalizedKeyComputer createNormalizedKeyComputer(Configuration conf) {
         Class<? extends NormalizedKeyComputer> nmkClass = getNormalizedKeyComputerClass(conf);
-        return ReflectionUtils.newInstance(nmkClass, conf);
+        try {
+            return nmkClass.newInstance();
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
     }
 
     /**
@@ -199,10 +214,14 @@
             Configuration conf) {
         List<Class<? extends GlobalAggregator<I, V, E, M, P, F>>> globalAggregatorClasses = getGlobalAggregatorClasses(conf);
         List<GlobalAggregator> aggs = new ArrayList<GlobalAggregator>();
-        for (Class<? extends GlobalAggregator<I, V, E, M, P, F>> globalAggClass : globalAggregatorClasses) {
-            aggs.add(ReflectionUtils.newInstance(globalAggClass, conf));
+        try {
+            for (Class<? extends GlobalAggregator<I, V, E, M, P, F>> globalAggClass : globalAggregatorClasses) {
+                aggs.add(globalAggClass.newInstance());
+            }
+            return aggs;
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
         }
-        return aggs;
     }
 
     /**
@@ -267,8 +286,13 @@
     public static <I extends WritableComparable, V extends Writable, E extends Writable, M extends WritableSizable> Vertex<I, V, E, M> createVertex(
             Configuration conf) {
         Class<? extends Vertex<I, V, E, M>> vertexClass = getVertexClass(conf);
-        Vertex<I, V, E, M> vertex = ReflectionUtils.newInstance(vertexClass, conf);
-        return vertex;
+        try {
+            Vertex<I, V, E, M> vertex = vertexClass.newInstance();
+            return vertex;
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+
     }
 
     /**
@@ -747,6 +771,76 @@
         return conf.getInt(PregelixJob.CKP_INTERVAL, -1);
     }
 
+    /**
+     * Get the grouping algorithm
+     * 
+     * @param conf
+     * @return true-sort; false-hash
+     */
+    public static boolean getGroupingAlgorithm(Configuration conf) {
+        return conf.getBoolean(PregelixJob.GROUPING_ALGORITHM, true);
+    }
+
+    /**
+     * Get the memory limit for the grouping algorithm (hash only)
+     * 
+     * @param conf
+     * @return the memory limit for hash-based grouping
+     */
+    public static int getGroupingMemoryLimit(Configuration conf) {
+        return conf.getInt(PregelixJob.GROUPING_MEM, 1000);
+    }
+
+    /**
+     * Get the memory limit for the sort algorithm
+     * 
+     * @param conf
+     * @return the memory limit for sorting
+     */
+    public static int getSortMemoryLimit(Configuration conf) {
+        return conf.getInt(PregelixJob.GROUPING_MEM, 1000);
+    }
+
+    /**
+     * Get the desired number of workers
+     * 
+     * @param conf
+     * @return the number of workers
+     */
+    public static int getNumberWorkers(Configuration conf) {
+        return conf.getInt(PregelixJob.NUM_WORKERS, -1);
+    }
+
+    /**
+     * Get whether the combiner key can be skipped when calling a user-defined combine function
+     * 
+     * @param conf
+     * @return true to skip; false otherwise
+     */
+    public static boolean getSkipCombinerKey(Configuration conf) {
+        return conf.getBoolean(PregelixJob.SKIP_COMBINER_KEY, false);
+    }
+
+    /**
+     * Get whether a merge connector is used
+     * 
+     * @param conf
+     * @return true -merge; false-no merge
+     */
+    public static boolean getMergingConnector(Configuration conf) {
+        return conf.getBoolean(PregelixJob.MERGE_CONNECTOR, true);
+    }
+
+    /**
+     * return the maximum iteration number
+     * 
+     * @param conf
+     * @return the maximum iteration number
+     */
+    public static int getMaxIteration(Configuration conf) {
+        return conf.getInt(PregelixJob.MAX_ITERATION, Integer.MAX_VALUE);
+    }
+
     public static Writable readGlobalAggregateValue(Configuration conf, String jobId, String aggClassName)
             throws HyracksDataException {
         try {
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultMessageCombiner.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultMessageCombiner.java
index feb9e2f..a05666e 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultMessageCombiner.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/DefaultMessageCombiner.java
@@ -92,4 +92,28 @@
         }
         return size;
     }
+
+    @Override
+    public int estimateAccumulatedStateByteSizePartial2(I vertexIndex, MsgList partialAggregate)
+            throws HyracksDataException {
+        int size = accumulatedSize;
+        for (int i = 0; i < partialAggregate.size(); i++) {
+            size += ((M) partialAggregate.get(i)).sizeInBytes();
+        }
+        return size;
+    }
+
+    @Override
+    public void stepPartial2(I vertexIndex, MsgList partialAggregate) throws HyracksDataException {
+        msgList.addAllElements(partialAggregate);
+        for (int i = 0; i < partialAggregate.size(); i++) {
+            accumulatedSize += ((M) partialAggregate.get(i)).sizeInBytes();
+        }
+    }
+
+    @Override
+    public MsgList finishPartial2() {
+        msgList.setSegmentEnd(false);
+        return msgList;
+    }
 }
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java
index 922920e..943a130 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java
@@ -28,6 +28,7 @@
 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.comm.util.FrameUtils;
+import edu.uci.ics.pregelix.api.io.Pointable;
 
 public class FrameTupleUtils {
 
@@ -42,6 +43,28 @@
         }
     }
 
+    public static void flushPointableKeyValueTuple(FrameTupleAppender appender, IFrameWriter writer, Pointable key,
+            Pointable value) throws HyracksDataException {
+        if (!flushPointableKeyValueTupleInternal(appender, key, value)) {
+            FrameUtils.flushFrame(appender.getBuffer(), writer);
+            appender.reset(appender.getBuffer(), true);
+            if (!flushPointableKeyValueTupleInternal(appender, key, value)) {
+                throw new IllegalStateException();
+            }
+        }
+    }
+
+    private static boolean flushPointableKeyValueTupleInternal(FrameTupleAppender appender, Pointable key,
+            Pointable value) {
+        if (!appender.appendField(key.getByteArray(), key.getStartOffset(), key.getLength())) {
+            return false;
+        }
+        if (!appender.appendField(value.getByteArray(), value.getStartOffset(), value.getLength())) {
+            return false;
+        }
+        return true;
+    }
+
     public static void flushTuplesFinal(FrameTupleAppender appender, IFrameWriter writer) throws HyracksDataException {
         if (appender.getTupleCount() > 0) {
             FrameUtils.flushFrame(appender.getBuffer(), writer);
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalCountAggregator.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalEdgeCountAggregator.java
similarity index 89%
copy from pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalCountAggregator.java
copy to pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalEdgeCountAggregator.java
index 9a95f09..f2610e7 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalCountAggregator.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalEdgeCountAggregator.java
@@ -24,7 +24,7 @@
 import edu.uci.ics.pregelix.api.io.WritableSizable;
 
 @SuppressWarnings("rawtypes")
-public class GlobalCountAggregator<I extends WritableComparable, V extends Writable, E extends Writable, M extends WritableSizable>
+public class GlobalEdgeCountAggregator<I extends WritableComparable, V extends Writable, E extends Writable, M extends WritableSizable>
         extends GlobalAggregator<I, V, E, M, LongWritable, LongWritable> {
 
     private LongWritable state = new LongWritable(0);
@@ -36,7 +36,7 @@
 
     @Override
     public void step(Vertex<I, V, E, M> v) throws HyracksDataException {
-        state.set(state.get() + 1);
+        state.set(state.get() + v.getEdges().size());
     }
 
     @Override
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalCountAggregator.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalVertexCountAggregator.java
similarity index 92%
rename from pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalCountAggregator.java
rename to pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalVertexCountAggregator.java
index 9a95f09..71e572f 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalCountAggregator.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/GlobalVertexCountAggregator.java
@@ -24,7 +24,7 @@
 import edu.uci.ics.pregelix.api.io.WritableSizable;
 
 @SuppressWarnings("rawtypes")
-public class GlobalCountAggregator<I extends WritableComparable, V extends Writable, E extends Writable, M extends WritableSizable>
+public class GlobalVertexCountAggregator<I extends WritableComparable, V extends Writable, E extends Writable, M extends WritableSizable>
         extends GlobalAggregator<I, V, E, M, LongWritable, LongWritable> {
 
     private LongWritable state = new LongWritable(0);
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/HadoopCountersAggregator.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/HadoopCountersAggregator.java
index b0814d9..45965c2 100644
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/HadoopCountersAggregator.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/HadoopCountersAggregator.java
@@ -22,9 +22,7 @@
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Counters;
 
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
-import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.io.WritableSizable;
 
 /**
diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ResetableByteArrayOutputStream.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ResetableByteArrayOutputStream.java
index 4b417be..abd9f00 100755
--- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ResetableByteArrayOutputStream.java
+++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ResetableByteArrayOutputStream.java
@@ -15,11 +15,8 @@
 package edu.uci.ics.pregelix.api.util;
 
 import java.io.OutputStream;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 public class ResetableByteArrayOutputStream extends OutputStream {
-    private static final Logger LOGGER = Logger.getLogger(ResetableByteArrayOutputStream.class.getName());
 
     private byte[] data;
     private int position;
@@ -34,22 +31,14 @@
 
     @Override
     public void write(int b) {
-        int remaining = data.length - position;
         if (position + 1 > data.length - 1)
             throw new IndexOutOfBoundsException();
         data[position] = (byte) b;
         position++;
-        if (LOGGER.isLoggable(Level.FINEST)) {
-            LOGGER.finest("write(): value: " + b + " remaining: " + remaining + " position: " + position);
-        }
     }
 
     @Override
     public void write(byte[] bytes, int offset, int length) {
-        if (LOGGER.isLoggable(Level.FINEST)) {
-            LOGGER.finest("write(bytes[], int, int) offset: " + offset + " length: " + length + " position: "
-                    + position);
-        }
         if (position + length > data.length - 1)
             throw new IndexOutOfBoundsException();
         System.arraycopy(bytes, offset, data, position, length);
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
index b290907..90cea28 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
@@ -20,44 +20,45 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.StringTokenizer;
 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.VLongWritable;
 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 TextCCInputFormat extends TextVertexInputFormat<LongWritable, LongWritable, NullWritable> {
+public class TextCCInputFormat extends TextVertexInputFormat<VLongWritable, VLongWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
         return new TextVertexReaderFromEachLine() {
-            String[] items;
+            StringTokenizer items;
 
             @Override
-            protected LongWritable getId(Text line) throws IOException {
-                items = line.toString().split(" ");
-                return new LongWritable(Long.parseLong(items[0]));
+            protected VLongWritable getId(Text line) throws IOException {
+                items = new StringTokenizer(line.toString());
+                return new VLongWritable(Long.parseLong(items.nextToken()));
             }
 
             @Override
-            protected LongWritable getValue(Text line) throws IOException {
+            protected VLongWritable 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);
+            protected Iterable<Edge<VLongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<VLongWritable, NullWritable>> edges = new ArrayList<Edge<VLongWritable, NullWritable>>();
+                Map<VLongWritable, NullWritable> edgeMap = new HashMap<VLongWritable, NullWritable>();
+                while (items.hasMoreTokens()) {
+                    edgeMap.put(new VLongWritable(Long.parseLong(items.nextToken())), null);
                 }
-                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
+                for (Entry<VLongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<VLongWritable, NullWritable> edge = new MapMutableEdge<VLongWritable, 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/TextCCOutputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java
index 770c6e1..8efff65 100644
--- 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
@@ -19,19 +19,19 @@
 
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.formats.TextVertexOutputFormat;
-import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.VLongWritable;
 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> {
+public class TextCCOutputFormat extends TextVertexOutputFormat<VLongWritable, VLongWritable, NullWritable> {
 
     @Override
     public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
         return new TextVertexWriterToEachLine() {
 
             @Override
-            protected Text convertVertexToLine(Vertex<LongWritable, LongWritable, NullWritable, ?> vertex)
+            protected Text convertVertexToLine(Vertex<VLongWritable, VLongWritable, 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/io/TextPRInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
index 38eef3a..24d1b5f 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
@@ -20,28 +20,29 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.StringTokenizer;
 
 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.LongWritable;
+import org.apache.hadoop.io.VLongWritable;
 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 TextPRInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable> {
+public class TextPRInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
         return new TextVertexReaderFromEachLine() {
-            String[] items;
+            StringTokenizer items;
 
             @Override
-            protected LongWritable getId(Text line) throws IOException {
-                items = line.toString().split(" ");
-                return new LongWritable(Long.parseLong(items[0]));
+            protected VLongWritable getId(Text line) throws IOException {
+                items = new StringTokenizer(line.toString());
+                return new VLongWritable(Long.parseLong(items.nextToken()));
             }
 
             @Override
@@ -50,14 +51,14 @@
             }
 
             @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);
+            protected Iterable<Edge<VLongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<VLongWritable, NullWritable>> edges = new ArrayList<Edge<VLongWritable, NullWritable>>();
+                Map<VLongWritable, NullWritable> edgeMap = new HashMap<VLongWritable, NullWritable>();
+                while (items.hasMoreTokens()) {
+                    edgeMap.put(new VLongWritable(Long.parseLong(items.nextToken())), null);
                 }
-                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
+                for (Entry<VLongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<VLongWritable, NullWritable> edge = new MapMutableEdge<VLongWritable, 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
index b14de6f..ba0eb9d 100644
--- 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
@@ -20,19 +20,19 @@
 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.VLongWritable;
 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> {
+public class TextPROutputFormat extends TextVertexOutputFormat<VLongWritable, DoubleWritable, NullWritable> {
 
     @Override
     public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
         return new TextVertexWriterToEachLine() {
 
             @Override
-            protected Text convertVertexToLine(Vertex<LongWritable, DoubleWritable, NullWritable, ?> vertex)
+            protected Text convertVertexToLine(Vertex<VLongWritable, 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/io/TextSPInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
index 953e93c..0201354 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
@@ -20,28 +20,29 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.StringTokenizer;
 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.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextSPInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
+public class TextSPInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, DoubleWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
         return new TextVertexReaderFromEachLine() {
-            String[] items;
+            StringTokenizer items;
 
             @Override
-            protected LongWritable getId(Text line) throws IOException {
-                items = line.toString().split(" ");
-                return new LongWritable(Long.parseLong(items[0]));
+            protected VLongWritable getId(Text line) throws IOException {
+                items = new StringTokenizer(line.toString());
+                return new VLongWritable(Long.parseLong(items.nextToken()));
             }
 
             @Override
@@ -50,14 +51,14 @@
             }
 
             @Override
-            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 LongWritable(Long.parseLong(items[i])), null);
+            protected Iterable<Edge<VLongWritable, DoubleWritable>> getEdges(Text line) throws IOException {
+                List<Edge<VLongWritable, DoubleWritable>> edges = new ArrayList<Edge<VLongWritable, DoubleWritable>>();
+                Map<VLongWritable, DoubleWritable> edgeMap = new HashMap<VLongWritable, DoubleWritable>();
+                while (items.hasMoreTokens()) {
+                    edgeMap.put(new VLongWritable(Long.parseLong(items.nextToken())), null);
                 }
-                for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
+                for (Entry<VLongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<VLongWritable, DoubleWritable> edge = new MapMutableEdge<VLongWritable, DoubleWritable>();
                     edge.setEntry(entry);
                     edge.setValue(new DoubleWritable(1.0));
                     edges.add(edge);
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
deleted file mode 100644
index 0a70b3c..0000000
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java
+++ /dev/null
@@ -1,72 +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.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/io2/TextPRInputFormat2.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
deleted file mode 100644
index 63a4519..0000000
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
+++ /dev/null
@@ -1,71 +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.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.DoubleWritable;
-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 TextPRInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, 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(items[0]));
-            }
-
-            @Override
-            protected DoubleWritable 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/io2/TextSPInputFormat2.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
deleted file mode 100644
index fdb1061..0000000
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
+++ /dev/null
@@ -1,72 +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.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.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-public class TextSPInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
-
-    @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 DoubleWritable getValue(Text line) throws IOException {
-                return null;
-            }
-
-            @Override
-            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 LongWritable(Long.parseLong(items[i])), null);
-                }
-                for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
-                    edge.setEntry(entry);
-                    edge.setValue(new DoubleWritable(1.0));
-                    edges.add(edge);
-                }
-                return edges;
-            }
-
-        };
-    }
-
-}
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
index 3789d6d..0c457c1 100644
--- 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
@@ -17,12 +17,13 @@
 
 import java.io.IOException;
 
+import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.VLongWritable;
 
-public class ConnectedComponentsVertex extends Vertex<LongWritable, LongWritable, NullWritable, LongWritable> {
+public class ConnectedComponentsVertex extends Vertex<VLongWritable, VLongWritable, NullWritable, VLongWritable> {
     /**
      * 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.
@@ -32,12 +33,12 @@
      * @throws IOException
      */
     @Override
-    public void compute(Iterable<LongWritable> messages) throws IOException {
+    public void compute(Iterable<VLongWritable> 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()) {
+            for (Edge<VLongWritable, NullWritable> edge : getEdges()) {
                 long neighbor = edge.getTargetVertexId().get();
                 if (neighbor < currentComponent) {
                     currentComponent = neighbor;
@@ -45,9 +46,9 @@
             }
             // 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();
+                setValue(new VLongWritable(currentComponent));
+                for (Edge<VLongWritable, NullWritable> edge : getEdges()) {
+                    VLongWritable neighbor = edge.getTargetVertexId();
                     if (neighbor.get() > currentComponent) {
                         sendMessage(neighbor, getValue());
                     }
@@ -60,7 +61,7 @@
 
         boolean changed = false;
         // did we get a smaller id ?
-        for (LongWritable message : messages) {
+        for (VLongWritable message : messages) {
             long candidateComponent = message.get();
             if (candidateComponent < currentComponent) {
                 currentComponent = candidateComponent;
@@ -70,9 +71,27 @@
 
         // propagate new component id to the neighbors
         if (changed) {
-            setValue(new LongWritable(currentComponent));
+            setValue(new VLongWritable(currentComponent));
             sendMessageToAllEdges(getValue());
         }
         voteToHalt();
     }
+
+    public static class MinCombiner extends Combiner<VLongWritable, VLongWritable> {
+
+        @Override
+        public void combine(VLongWritable vertexIndex, VLongWritable originalMessage, VLongWritable messageToCombine) {
+            long oldValue = messageToCombine.get();
+            long newValue = originalMessage.get();
+            if (newValue < oldValue) {
+                messageToCombine.set(newValue);
+            }
+        }
+
+        @Override
+        public VLongWritable createInitialMessage() {
+            return new VLongWritable(Integer.MAX_VALUE);
+        }
+
+    }
 }
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
index 86e90dd..38c4ad9 100644
--- 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
@@ -1,11 +1,14 @@
 /*
- * 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
- * 
+ * 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.
@@ -15,34 +18,51 @@
 
 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 java.io.IOException;
+
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.VLongWritable;
 
 /**
- * The PageRank algorithm, with uniform transition probabilities on the edges
- * http://en.wikipedia.org/wiki/PageRank
+ * Implementation of PageRank in which vertex ids are ints, page rank values
+ * are floats, and graph is unweighted.
  */
-public class PageRankVertex extends RandomWalkVertex<NullWritable> {
+public class PageRankVertex extends Vertex<VLongWritable, DoubleWritable, NullWritable, DoubleWritable> {
+    /** Number of supersteps */
+    public static final int maxSuperStep = 4;
 
     @Override
-    protected double transitionProbability(double stateProbability, Edge<LongWritable, NullWritable> edge) {
-        return stateProbability / getNumEdges();
+    public void compute(Iterable<DoubleWritable> messages) throws IOException {
+        if (getSuperstep() >= 1) {
+            float sum = 0;
+            for (DoubleWritable message : messages) {
+                sum += message.get();
+            }
+            getValue().set((0.15f / getTotalNumVertices()) + 0.85f * sum);
+        }
+
+        if (getSuperstep() < maxSuperStep) {
+            sendMessageToAllEdges(new DoubleWritable(getValue().get() / getNumEdges()));
+        } else {
+            voteToHalt();
+        }
     }
 
-    @Override
-    protected double recompute(Iterable<DoubleWritable> partialRanks, double teleportationProbability) {
+    public static class SumCombiner extends Combiner<VLongWritable, DoubleWritable> {
 
-        // rank contribution from incident neighbors
-        double rankFromNeighbors = MathUtils.sum(partialRanks);
-        // rank contribution from dangling vertices
-        double danglingContribution = getDanglingProbability() / getTotalNumVertices();
+        @Override
+        public void combine(VLongWritable vertexIndex, DoubleWritable originalMessage, DoubleWritable messageToCombine) {
+            double oldValue = messageToCombine.get();
+            messageToCombine.set(oldValue + originalMessage.get());
+        }
 
-        // recompute rank
-        return (1d - teleportationProbability) * (rankFromNeighbors + danglingContribution) + teleportationProbability
-                / getTotalNumVertices();
+        @Override
+        public DoubleWritable createInitialMessage() {
+            return new DoubleWritable(0.0);
+        }
+
     }
-}
+}
\ No newline at end of file
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
index 755a3d0..935df53 100644
--- 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
@@ -20,15 +20,16 @@
 
 import java.io.IOException;
 
+import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.VLongWritable;
 
 /**
  * Shortest paths algorithm.
  */
-public class ShortestPathsVertex extends Vertex<LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> {
+public class ShortestPathsVertex extends Vertex<VLongWritable, DoubleWritable, DoubleWritable, DoubleWritable> {
     /** Source id. */
     public static final String SOURCE_ID = "giraph.shortestPathsBenchmark.sourceId";
     /** Default source id. */
@@ -51,7 +52,7 @@
 
         if (minDist < getValue().get()) {
             setValue(new DoubleWritable(minDist));
-            for (Edge<LongWritable, DoubleWritable> edge : getEdges()) {
+            for (Edge<VLongWritable, DoubleWritable> edge : getEdges()) {
                 double distance = minDist + edge.getValue().get();
                 sendMessage(edge.getTargetVertexId(), new DoubleWritable(distance));
             }
@@ -59,4 +60,22 @@
 
         voteToHalt();
     }
+
+    public static class MinCombiner extends Combiner<VLongWritable, DoubleWritable> {
+
+        @Override
+        public void combine(VLongWritable vertexIndex, DoubleWritable originalMessage, DoubleWritable messageToCombine) {
+            double oldValue = messageToCombine.get();
+            double newValue = originalMessage.get();
+            if (newValue < oldValue) {
+                messageToCombine.set(newValue);
+            }
+        }
+
+        @Override
+        public DoubleWritable createInitialMessage() {
+            return new DoubleWritable(Integer.MAX_VALUE);
+        }
+
+    }
 }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IDriver.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IDriver.java
index c72f392..dd38425 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IDriver.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IDriver.java
@@ -24,9 +24,7 @@
 
     public static enum Plan {
         INNER_JOIN,
-        OUTER_JOIN,
-        OUTER_JOIN_SORT,
-        OUTER_JOIN_SINGLE_SORT
+        OUTER_JOIN
     }
 
     public void runJob(PregelixJob job, String ipAddress, int port) throws HyracksException;
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IJobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IJobGen.java
index 6bb0dea..b5074a6 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IJobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/base/IJobGen.java
@@ -30,6 +30,6 @@
 
     public JobSpecification[] generateLoadingCheckpoint(int lastCheckpointedIteration) throws HyracksException;
 
-    public JobSpecification generateClearState() throws HyracksException;
+    public JobSpecification generateClearState(boolean allStates) throws HyracksException;
 
 }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
index a71ea3d..3d5f5cd 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
@@ -18,6 +18,7 @@
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
+import java.lang.reflect.Type;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.ArrayList;
@@ -30,9 +31,12 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
@@ -45,10 +49,16 @@
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.client.stats.Counters;
 import edu.uci.ics.hyracks.client.stats.impl.ClientCounterContext;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.MessageCombiner;
+import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.job.ICheckpointHook;
 import edu.uci.ics.pregelix.api.job.IIterationCompleteReporterHook;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.api.util.GlobalEdgeCountAggregator;
+import edu.uci.ics.pregelix.api.util.GlobalVertexCountAggregator;
+import edu.uci.ics.pregelix.api.util.ReflectionUtils;
 import edu.uci.ics.pregelix.core.base.IDriver;
 import edu.uci.ics.pregelix.core.jobgen.JobGen;
 import edu.uci.ics.pregelix.core.jobgen.JobGenFactory;
@@ -67,6 +77,7 @@
     private IHyracksClientConnection hcc;
     private Class exampleClass;
     private boolean profiling = false;
+    private StringBuffer counterBuffer = new StringBuffer();
 
     public Driver(Class exampleClass) {
         this.exampleClass = exampleClass;
@@ -92,9 +103,14 @@
     public void runJobs(List<PregelixJob> jobs, Plan planChoice, String ipAddress, int port, boolean profiling)
             throws HyracksException {
         try {
+            counterBuffer.delete(0, counterBuffer.length());
+            counterBuffer.append("performance counters\n");
             if (jobs.size() <= 0) {
                 throw new HyracksException("Please submit at least one job for execution!");
             }
+            for (PregelixJob job : jobs) {
+                initJobConfiguration(job);
+            }
             this.profiling = profiling;
             PregelixJob currentJob = jobs.get(0);
             PregelixJob lastJob = currentJob;
@@ -129,16 +145,22 @@
                         addHadoopConfiguration(currentJob, ipAddress, port, failed);
                         ICheckpointHook ckpHook = BspUtils.createCheckpointHook(currentJob.getConfiguration());
 
+                        boolean compatible = i == 0 ? false : compatible(lastJob, currentJob);
                         /** load the data */
-                        if ((i == 0 || compatible(lastJob, currentJob)) && !failed) {
-                            if (i != 0) {
+                        if (!failed) {
+                            if (i == 0) {
+                                jobGen.reset(currentJob);
+                                loadData(currentJob, jobGen, deploymentId);
+                            } else if (!compatible) {
                                 finishJobs(jobGen, deploymentId);
                                 /** invalidate/clear checkpoint */
                                 lastSnapshotJobIndex.set(0);
                                 lastSnapshotSuperstep.set(0);
+                                jobGen.reset(currentJob);
+                                loadData(currentJob, jobGen, deploymentId);
+                            } else {
+                                jobGen.reset(currentJob);
                             }
-                            jobGen.reset(currentJob);
-                            loadData(currentJob, jobGen, deploymentId);
                         } else {
                             jobGen.reset(currentJob);
                         }
@@ -147,14 +169,19 @@
                         jobGen = dynamicOptimizer.optimize(jobGen, i);
                         runLoopBody(deploymentId, currentJob, jobGen, i, lastSnapshotJobIndex, lastSnapshotSuperstep,
                                 ckpHook, failed);
-                        runClearState(deploymentId, jobGen);
                         failed = false;
                     }
 
                     /** finish the jobs */
                     finishJobs(jobGen, deploymentId);
+
                     /** clear checkpoints if any */
                     jobGen.clearCheckpoints();
+
+                    /** clear state */
+                    runClearState(deploymentId, jobGen, true);
+
+                    /** undeploy the binary */
                     hcc.unDeployBinary(deploymentId);
                 } catch (Exception e1) {
                     Set<String> blackListNodes = new HashSet<String>();
@@ -169,8 +196,6 @@
                 }
             } while (failed && retryCount < maxRetryCount);
             LOG.info("job finished");
-            StringBuffer counterBuffer = new StringBuffer();
-            counterBuffer.append("performance counters\n");
             for (String counter : COUNTERS) {
                 counterBuffer.append("\t" + counter + ": " + counterContext.getCounter(counter, false).get() + "\n");
             }
@@ -273,10 +298,9 @@
         if (doRecovery) {
             /** reload the checkpoint */
             if (snapshotSuperstep.get() > 0) {
-                runClearState(deploymentId, jobGen);
                 runLoadCheckpoint(deploymentId, jobGen, snapshotSuperstep.get());
             } else {
-                runClearState(deploymentId, jobGen);
+                runClearState(deploymentId, jobGen, true);
                 loadData(job, jobGen, deploymentId);
             }
         }
@@ -293,8 +317,21 @@
             end = System.currentTimeMillis();
             time = end - start;
             LOG.info(job + ": iteration " + i + " finished " + time + "ms");
+            if (i == 1) {
+                counterBuffer.append("\t"
+                        + "total vertice: "
+                        + IterationUtils.readGlobalAggregateValue(job.getConfiguration(),
+                                BspUtils.getJobId(job.getConfiguration()), GlobalVertexCountAggregator.class.getName())
+                        + "\n");
+                counterBuffer.append("\t"
+                        + "total edges: "
+                        + IterationUtils.readGlobalAggregateValue(job.getConfiguration(),
+                                BspUtils.getJobId(job.getConfiguration()), GlobalEdgeCountAggregator.class.getName())
+                        + "\n");
+            }
             terminate = IterationUtils.readTerminationState(job.getConfiguration(), jobGen.getJobId())
-                    || IterationUtils.readForceTerminationState(job.getConfiguration(), jobGen.getJobId());
+                    || IterationUtils.readForceTerminationState(job.getConfiguration(), jobGen.getJobId())
+                    || i >= BspUtils.getMaxIteration(job.getConfiguration());
             if (ckpHook.checkpoint(i) || (ckpInterval > 0 && i % ckpInterval == 0)) {
                 runCheckpoint(deploymentId, jobGen, i);
                 snapshotJobIndex.set(currentJobIndex);
@@ -369,9 +406,9 @@
         }
     }
 
-    private void runClearState(DeploymentId deploymentId, JobGen jobGen) throws Exception {
+    private void runClearState(DeploymentId deploymentId, JobGen jobGen, boolean allStates) throws Exception {
         try {
-            JobSpecification clear = jobGen.generateClearState();
+            JobSpecification clear = jobGen.generateClearState(allStates);
             execute(deploymentId, clear);
         } catch (Exception e) {
             throw e;
@@ -386,6 +423,7 @@
 
     private void execute(DeploymentId deploymentId, JobSpecification job) throws Exception {
         job.setUseConnectorPolicyForScheduling(false);
+        job.setReportTaskDetails(false);
         job.setMaxReattempts(0);
         JobId jobId = hcc.startJob(deploymentId, job,
                 profiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
@@ -403,6 +441,42 @@
         LOG.info("jar deployment finished " + (end - start) + "ms");
         return deploymentId;
     }
+
+    @SuppressWarnings({ "unchecked" })
+    private void initJobConfiguration(PregelixJob job) {
+        Configuration conf = job.getConfiguration();
+        Class vertexClass = conf.getClass(PregelixJob.VERTEX_CLASS, Vertex.class);
+        List<Type> parameterTypes = ReflectionUtils.getTypeArguments(Vertex.class, vertexClass);
+        Type vertexIndexType = parameterTypes.get(0);
+        Type vertexValueType = parameterTypes.get(1);
+        Type edgeValueType = parameterTypes.get(2);
+        Type messageValueType = parameterTypes.get(3);
+        conf.setClass(PregelixJob.VERTEX_INDEX_CLASS, (Class<?>) vertexIndexType, WritableComparable.class);
+        conf.setClass(PregelixJob.VERTEX_VALUE_CLASS, (Class<?>) vertexValueType, Writable.class);
+        conf.setClass(PregelixJob.EDGE_VALUE_CLASS, (Class<?>) edgeValueType, Writable.class);
+        conf.setClass(PregelixJob.MESSAGE_VALUE_CLASS, (Class<?>) messageValueType, Writable.class);
+
+        List aggregatorClasses = BspUtils.getGlobalAggregatorClasses(conf);
+        for (int i = 0; i < aggregatorClasses.size(); i++) {
+            Class aggregatorClass = (Class) aggregatorClasses.get(i);
+            if (!aggregatorClass.equals(GlobalAggregator.class)) {
+                List<Type> argTypes = ReflectionUtils.getTypeArguments(GlobalAggregator.class, aggregatorClass);
+                Type partialAggregateValueType = argTypes.get(4);
+                conf.setClass(PregelixJob.PARTIAL_AGGREGATE_VALUE_CLASS + "$" + aggregatorClass.getName(),
+                        (Class<?>) partialAggregateValueType, Writable.class);
+                Type finalAggregateValueType = argTypes.get(5);
+                conf.setClass(PregelixJob.FINAL_AGGREGATE_VALUE_CLASS + "$" + aggregatorClass.getName(),
+                        (Class<?>) finalAggregateValueType, Writable.class);
+            }
+        }
+
+        Class combinerClass = BspUtils.getMessageCombinerClass(conf);
+        if (!combinerClass.equals(MessageCombiner.class)) {
+            List<Type> argTypes = ReflectionUtils.getTypeArguments(MessageCombiner.class, combinerClass);
+            Type partialCombineValueType = argTypes.get(2);
+            conf.setClass(PregelixJob.PARTIAL_COMBINE_VALUE_CLASS, (Class<?>) partialCombineValueType, Writable.class);
+        }
+    }
 }
 
 class FileFilter implements FilenameFilter {
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
index 1600ab5..fef56d2 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/hadoop/config/ConfigurationFactory.java
@@ -25,6 +25,7 @@
 public class ConfigurationFactory implements IConfigurationFactory {
     private static final long serialVersionUID = 1L;
     private final byte[] data;
+    private transient Configuration confCache;
 
     public ConfigurationFactory(Configuration conf) {
         try {
@@ -35,22 +36,30 @@
     }
 
     @Override
-    public Configuration createConfiguration(IHyracksTaskContext ctx) throws HyracksDataException {
+    public synchronized Configuration createConfiguration(IHyracksTaskContext ctx) throws HyracksDataException {
         try {
+            if (confCache != null) {
+                return confCache;
+            }
             Configuration conf = new Configuration();
             conf.setClassLoader(ctx.getJobletContext().getClassLoader());
             SerDeUtils.deserialize(conf, data);
+            confCache = conf;
             return conf;
         } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
-    
+
     @Override
-    public Configuration createConfiguration() throws HyracksDataException{
+    public synchronized Configuration createConfiguration() throws HyracksDataException {
         try {
+            if (confCache != null) {
+                return confCache;
+            }
             Configuration conf = new Configuration();
             SerDeUtils.deserialize(conf, data);
+            confCache = conf;
             return conf;
         } catch (Exception e) {
             throw new HyracksDataException(e);
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 109a91a..c1fb82c 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
@@ -66,6 +66,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 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.group.HashSpillableTableFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 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;
@@ -119,6 +122,9 @@
 import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
 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.group.ClusteredGroupOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.group.IClusteredAggregatorDescriptorFactory;
+import edu.uci.ics.pregelix.dataflow.std.sort.FastSortOperatorDescriptor;
 import edu.uci.ics.pregelix.runtime.bootstrap.IndexLifeCycleManagerProvider;
 import edu.uci.ics.pregelix.runtime.bootstrap.StorageManagerInterface;
 import edu.uci.ics.pregelix.runtime.bootstrap.VirtualBufferCacheProvider;
@@ -131,17 +137,19 @@
 
 public abstract class JobGen implements IJobGen {
     private static final Logger LOGGER = Logger.getLogger(JobGen.class.getName());
+    protected static final int BF_HINT=100000;
     protected static final int MB = 1048576;
     protected static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
-    protected static final int tableSize = 10485767;
+    protected static final int tableSize = 1575767;
     protected static final String PRIMARY_INDEX = "primary";
     protected Configuration conf;
+    protected IConfigurationFactory confFactory;
     protected PregelixJob pregelixJob;
     protected IIndexLifecycleManagerProvider lcManagerProvider = IndexLifeCycleManagerProvider.INSTANCE;
     protected IStorageManagerInterface storageManagerInterface = StorageManagerInterface.INSTANCE;
     protected String jobId = UUID.randomUUID().toString();;
     protected int frameSize = ClusterConfig.getFrameSize();
-    protected int maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
+    protected int maxFrameNumber = (int) (((long) 64 * MB) / frameSize);
     protected IOptimizer optimizer;
 
     private static final Map<String, String> MERGE_POLICY_PROPERTIES;
@@ -168,17 +176,17 @@
         this.optimizer = optimizer;
         conf = job.getConfiguration();
         pregelixJob = job;
-        initJobConfiguration();
         job.setJobId(jobId);
         // set the frame size to be the one user specified if the user did specify.
         int specifiedFrameSize = BspUtils.getFrameSize(job.getConfiguration());
         if (specifiedFrameSize > 0) {
             frameSize = specifiedFrameSize;
-            maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
+            maxFrameNumber = BspUtils.getSortMemoryLimit(conf);
         }
         if (maxFrameNumber <= 0) {
-            maxFrameNumber = 1;
+            maxFrameNumber = 1000;
         }
+        initJobConfiguration();
     }
 
     public void reset(PregelixJob job) {
@@ -218,6 +226,7 @@
             Type partialCombineValueType = argTypes.get(2);
             conf.setClass(PregelixJob.PARTIAL_COMBINE_VALUE_CLASS, (Class<?>) partialCombineValueType, Writable.class);
         }
+        this.confFactory = new ConfigurationFactory(conf);
     }
 
     public String getJobId() {
@@ -277,7 +286,7 @@
         }
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory confFactory = getConfigurationFactory();
         String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits);
         VertexFileScanOperatorDescriptor scanner = new VertexFileScanOperatorDescriptor(spec, recordDescriptor, splits,
                 readSchedule, confFactory);
@@ -304,7 +313,7 @@
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), vertexClass.getName());
         VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
                 resultFileSplitProvider, preHookFactory, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
@@ -345,7 +354,7 @@
         /**
          * construct btree search operator
          */
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory confFactory = getConfigurationFactory();
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
@@ -368,7 +377,7 @@
         IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results);
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), vertexClass.getName());
         VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory,
                 resultFileSplitProvider, preHookFactory, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" });
@@ -464,9 +473,9 @@
     /***
      * generate a "clear state" job
      */
-    public JobSpecification generateClearState() throws HyracksException {
-        JobSpecification spec = new JobSpecification(frameSize);
-        ClearStateOperatorDescriptor clearState = new ClearStateOperatorDescriptor(spec, jobId);
+    public JobSpecification generateClearState(boolean allStates) throws HyracksException {
+        JobSpecification spec = new JobSpecification();
+        ClearStateOperatorDescriptor clearState = new ClearStateOperatorDescriptor(spec, jobId, allStates);
         setLocationConstraint(spec, clearState);
         spec.addRoot(clearState);
         return spec;
@@ -493,7 +502,7 @@
 
     @SuppressWarnings({ "unchecked", "rawtypes" })
     protected ITuplePartitionComputerFactory getVertexPartitionComputerFactory() {
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory confFactory = getConfigurationFactory();
         Class<? extends VertexPartitioner> partitionerClazz = BspUtils.getVertexPartitionerClass(conf);
         if (partitionerClazz != null) {
             return new VertexPartitionComputerFactory(confFactory);
@@ -567,7 +576,7 @@
         typeTraits[1] = new TypeTraits(false);
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                sortFields, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, true, 0, false,
+                sortFields, fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, true, BF_HINT, false,
                 getIndexDataflowHelperFactory(), NoOpOperationCallbackFactory.INSTANCE);
         setLocationConstraint(spec, btreeBulkLoad);
 
@@ -638,7 +647,7 @@
          */
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), vertexClass.getName());
         VertexFileWriteOperatorDescriptor writer = new VertexFileWriteOperatorDescriptor(spec, confFactory,
                 inputRdFactory, preHookFactory);
         setLocationConstraint(spec, writer);
@@ -703,7 +712,7 @@
         tmpJob.setOutputValueClass(MsgList.class);
 
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), MsgList.class.getName());
+                new ConfigurationFactory(tmpJob.getConfiguration()), vertexIdClass.getName(), MsgList.class.getName());
         HDFSFileWriteOperatorDescriptor hdfsWriter = new HDFSFileWriteOperatorDescriptor(spec, tmpJob, inputRdFactory);
         setLocationConstraint(spec, hdfsWriter);
 
@@ -769,7 +778,7 @@
         /** construct runtime hook */
         RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
                 new RecoveryRuntimeHookFactory(jobId, lastCheckpointedIteration, new ConfigurationFactory(
-                        pregelixJob.getConfiguration())));
+                        tmpJob.getConfiguration())));
         setLocationConstraint(spec, postSuperStep);
 
         /** construct empty sink operator */
@@ -846,10 +855,10 @@
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory configurationFactory = getConfigurationFactory();
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), vertexClass.getName());
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
         TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
@@ -902,4 +911,119 @@
         return optimizer.getOptimizedFileSplitProvider(jobId, indexName);
     }
 
+    /**
+     * @return the PregelixJob configuration
+     */
+    public PregelixJob getPregelixJob() {
+        return pregelixJob;
+    }
+
+    /**
+     * Generate the pipeline for local grouping
+     * 
+     * @param spec
+     *            the JobSpecification
+     * @param sortOrHash
+     *            sort-based algorithm or hash-based algorithm
+     * @return the start and end (if any) operators of the grouping pipeline
+     */
+    protected Pair<IOperatorDescriptor, IOperatorDescriptor> generateGroupingOperators(JobSpecification spec,
+            int iteration, Class<? extends Writable> vertexIdClass) throws HyracksException {
+        int[] keyFields = new int[] { 0 };
+        Class<? extends Writable> messageValueClass = BspUtils.getMessageValueClass(conf);
+        Class<? extends Writable> partialCombineValueClass = BspUtils.getPartialCombineValueClass(conf);
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
+        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
+        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
+                vertexIdClass.getName(), messageValueClass.getName());
+        RecordDescriptor rdCombinedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
+                vertexIdClass.getName(), partialCombineValueClass.getName());
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
+                MsgList.class.getName());
+        boolean sortOrHash = BspUtils.getGroupingAlgorithm(conf);
+        boolean merge = BspUtils.getMergingConnector(conf);
+
+        if (sortOrHash) {
+            /**
+             * construct local sort operator
+             */
+            IClusteredAggregatorDescriptorFactory localAggregatorFactory = DataflowUtils
+                    .getAccumulatingAggregatorFactory(this.getConfigurationFactory(), false, false);
+            IClusteredAggregatorDescriptorFactory partialAggregatorFactory = DataflowUtils
+                    .getAccumulatingAggregatorFactory(this.getConfigurationFactory(), false, true);
+            IOperatorDescriptor localGby = new FastSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
+                    rdUnnestedMessage, keyFields, localAggregatorFactory, partialAggregatorFactory, rdCombinedMessage,
+                    rdCombinedMessage, true);
+            setLocationConstraint(spec, localGby);
+
+            /**
+             * construct global group-by operator
+             */
+            IClusteredAggregatorDescriptorFactory finalAggregatorFactory = DataflowUtils
+                    .getAccumulatingAggregatorFactory(getConfigurationFactory(), true, true);
+            ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
+            if (merge) {
+                IOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields, sortCmpFactories,
+                        finalAggregatorFactory, rdFinal);
+                setLocationConstraint(spec, globalGby);
+                spec.connect(
+                        new edu.uci.ics.pregelix.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor(
+                                spec, partionFactory, keyFields), localGby, 0, globalGby, 0);
+                return Pair.of(localGby, globalGby);
+            } else {
+                IOperatorDescriptor globalGby = new FastSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
+                        rdCombinedMessage, keyFields, partialAggregatorFactory, finalAggregatorFactory,
+                        rdCombinedMessage, rdFinal, false);
+                setLocationConstraint(spec, globalGby);
+                spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalGby, 0);
+                return Pair.of(localGby, globalGby);
+            }
+        } else {
+            int frameLimit = BspUtils.getGroupingMemoryLimit(conf);
+            int hashTableSize = Math.round(((float) frameLimit / 1000f) * tableSize);
+            /**
+             * construct local group-by operator
+             */
+            ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
+            IAggregatorDescriptorFactory localAggregatorFactory = DataflowUtils.getSerializableAggregatorFactory(
+                    getConfigurationFactory(), false, false);
+            IAggregatorDescriptorFactory partialAggregatorFactory = DataflowUtils.getSerializableAggregatorFactory(
+                    getConfigurationFactory(), false, true);
+            IOperatorDescriptor localGby = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimit,
+                    sortCmpFactories, nkmFactory, localAggregatorFactory, partialAggregatorFactory, rdUnnestedMessage,
+                    new HashSpillableTableFactory(partionFactory, hashTableSize), merge ? true : false);
+            setLocationConstraint(spec, localGby);
+
+            IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils
+                    .getAccumulatingAggregatorFactory(getConfigurationFactory(), true, true);
+            /**
+             * construct global group-by operator
+             */
+            if (merge) {
+                IOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields, sortCmpFactories,
+                        aggregatorFactoryFinal, rdFinal);
+                setLocationConstraint(spec, globalGby);
+
+                spec.connect(
+                        new edu.uci.ics.pregelix.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor(
+                                spec, partionFactory, keyFields), localGby, 0, globalGby, 0);
+                return Pair.of(localGby, globalGby);
+            } else {
+                IAggregatorDescriptorFactory finalAggregatorFactory = DataflowUtils.getSerializableAggregatorFactory(
+                        getConfigurationFactory(), true, true);
+                IOperatorDescriptor globalGby = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimit,
+                        sortCmpFactories, nkmFactory, partialAggregatorFactory, finalAggregatorFactory,
+                        rdCombinedMessage, new HashSpillableTableFactory(partionFactory, hashTableSize), false);
+                setLocationConstraint(spec, globalGby);
+
+                spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalGby, 0);
+                return Pair.of(localGby, globalGby);
+            }
+        }
+    }
+
+    public IConfigurationFactory getConfigurationFactory() {
+        return confFactory;
+    }
 }
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenFactory.java
index cbc9c81..ca1e227 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenFactory.java
@@ -30,12 +30,6 @@
             case OUTER_JOIN:
                 jobGen = new JobGenOuterJoin(currentJob, optimizer);
                 break;
-            case OUTER_JOIN_SORT:
-                jobGen = new JobGenOuterJoinSort(currentJob, optimizer);
-                break;
-            case OUTER_JOIN_SINGLE_SORT:
-                jobGen = new JobGenOuterJoinSingleSort(currentJob, optimizer);
-                break;
             default:
                 jobGen = new JobGenInnerJoin(currentJob, optimizer);
         }
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 a728d48..8122648 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
@@ -21,6 +21,7 @@
 import java.util.List;
 import java.util.logging.Logger;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.io.Writable;
@@ -34,6 +35,7 @@
 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -46,7 +48,6 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-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.misc.ConstantTupleSourceOperatorDescriptor;
@@ -61,7 +62,6 @@
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
 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.optimizer.IOptimizer;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
@@ -75,8 +75,6 @@
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.group.ClusteredGroupOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.group.IClusteredAggregatorDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
@@ -97,7 +95,7 @@
     public JobGenInnerJoin(PregelixJob job, IOptimizer optimizer) {
         super(job, optimizer);
     }
-    
+
     public JobGenInnerJoin(PregelixJob job, String jobId, IOptimizer optimizer) {
         super(job, jobId, optimizer);
     }
@@ -107,7 +105,8 @@
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
         Class<? extends Writable> messageValueClass = BspUtils.getMessageValueClass(conf);
         String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf);
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+
+        IConfigurationFactory confFactory = getConfigurationFactory();
         JobSpecification spec = new JobSpecification(frameSize);
 
         /**
@@ -143,10 +142,10 @@
                 VLongWritable.class.getName());
         RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
                 partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory configurationFactory = getConfigurationFactory();
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), vertexClass.getName());
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 MsgList.class.getName());
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
@@ -173,7 +172,7 @@
          * final aggregate write operator
          */
         IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
+                getConfigurationFactory(), partialAggregateValueClassNames);
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -192,33 +191,12 @@
         setLocationConstraint(spec, btreeBulkLoad);
 
         /**
-         * construct local sort operator
+         * construct group-by operator pipeline
          */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, localSort);
-
-        /**
-         * construct local pre-clustered group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf,
-                false, false);
-        ClusteredGroupOperatorDescriptor localGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
-        setLocationConstraint(spec, localGby);
-
-        /**
-         * construct global group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, true);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
+        Pair<IOperatorDescriptor, IOperatorDescriptor> groupOps = generateGroupingOperators(spec, iteration,
+                vertexIdClass);
+        IOperatorDescriptor groupStartOperator = groupOps.getLeft();
+        IOperatorDescriptor groupEndOperator = groupOps.getRight();
 
         /**
          * construct the materializing write operator
@@ -271,7 +249,7 @@
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, localSort, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, groupStartOperator, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
@@ -286,10 +264,7 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         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,
-                nkmFactory), localGby, 0, globalGby, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), groupEndOperator, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
 
@@ -336,7 +311,7 @@
         /**
          * construct pre-superstep
          */
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory confFactory = getConfigurationFactory();
         RuntimeHookOperatorDescriptor preSuperStep = new RuntimeHookOperatorDescriptor(spec,
                 new PreSuperStepRuntimeHookFactory(jobId, confFactory));
         setLocationConstraint(spec, preSuperStep);
@@ -370,10 +345,11 @@
                 VLongWritable.class.getName());
         RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
                 partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory configurationFactory = getConfigurationFactory();
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(),
+                vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
@@ -398,33 +374,12 @@
         setLocationConstraint(spec, btreeBulkLoad);
 
         /**
-         * construct local sort operator
+         * construct group-by operator pipeline
          */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, localSort);
-
-        /**
-         * construct local pre-clustered group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf,
-                false, false);
-        ClusteredGroupOperatorDescriptor localGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
-        setLocationConstraint(spec, localGby);
-
-        /**
-         * construct global group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, true);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
+        Pair<IOperatorDescriptor, IOperatorDescriptor> groupOps = generateGroupingOperators(spec, iteration,
+                vertexIdClass);
+        IOperatorDescriptor groupStartOperator = groupOps.getLeft();
+        IOperatorDescriptor groupEndOperator = groupOps.getRight();
 
         /**
          * construct the materializing write operator
@@ -453,7 +408,7 @@
          * final aggregate write operator
          */
         IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
+                getConfigurationFactory(), partialAggregateValueClassNames);
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -492,7 +447,7 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, setUnion, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), setUnion, 0, join, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, groupStartOperator, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
@@ -507,10 +462,7 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         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,
-                nkmFactory), localGby, 0, globalGby, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), groupEndOperator, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
 
@@ -688,7 +640,7 @@
          * construct write file operator
          */
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), MsgList.class.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), MsgList.class.getName());
         HDFSFileWriteOperatorDescriptor writer = new HDFSFileWriteOperatorDescriptor(spec, job, inputRdFactory);
         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 2853fd0..b4a12b8 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
@@ -14,13 +14,14 @@
  */
 package edu.uci.ics.pregelix.core.jobgen;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-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.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -28,11 +29,8 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-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;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -40,7 +38,6 @@
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.core.data.TypeTraits;
-import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.optimizer.IOptimizer;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
@@ -51,8 +48,6 @@
 import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.group.ClusteredGroupOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.group.IClusteredAggregatorDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
@@ -83,9 +78,10 @@
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
         Class<? extends Writable> messageValueClass = BspUtils.getMessageValueClass(conf);
         String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf);
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
-        JobSpecification spec = new JobSpecification(frameSize);
 
+        IConfigurationFactory confFactory = getConfigurationFactory();
+        JobSpecification spec = new JobSpecification(frameSize);
+	
         /**
          * construct empty tuple operator
          */
@@ -114,15 +110,17 @@
                 VLongWritable.class.getName());
         RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
                 partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory configurationFactory = getConfigurationFactory();
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), vertexClass.getName());
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
                 vertexIdClass.getName(), messageValueClass.getName());
         RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
                 vertexClass.getName());
         RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
+        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
+                MsgList.class.getName());
 
         TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
                 recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
@@ -132,35 +130,12 @@
         setLocationConstraint(spec, scanner);
 
         /**
-         * construct local sort operator
+         * construct group-by operator pipeline
          */
-        int[] keyFields = new int[] { 0 };
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, localSort);
-
-        /**
-         * construct local pre-clustered group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf,
-                false, false);
-        ClusteredGroupOperatorDescriptor localGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
-        setLocationConstraint(spec, localGby);
-
-        /**
-         * construct global group-by operator
-         */
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                MsgList.class.getName());
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, true);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
+        Pair<IOperatorDescriptor, IOperatorDescriptor> groupOps = generateGroupingOperators(spec, iteration,
+                vertexIdClass);
+        IOperatorDescriptor groupStartOperator = groupOps.getLeft();
+        IOperatorDescriptor groupEndOperator = groupOps.getRight();
 
         /**
          * construct the materializing write operator
@@ -189,7 +164,7 @@
          * final aggregate write operator
          */
         IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
+                getConfigurationFactory(), partialAggregateValueClassNames);
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -225,7 +200,7 @@
         /** connect all operators **/
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, localSort, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, groupStartOperator, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
@@ -242,10 +217,7 @@
         /**
          * connect the group-by operator
          */
-        spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 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), groupEndOperator, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink2, 0);
 
@@ -291,7 +263,7 @@
         /**
          * construct pre-superstep hook
          */
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory confFactory = getConfigurationFactory();
         RuntimeHookOperatorDescriptor preSuperStep = new RuntimeHookOperatorDescriptor(spec,
                 new PreSuperStepRuntimeHookFactory(jobId, confFactory));
         setLocationConstraint(spec, preSuperStep);
@@ -318,10 +290,11 @@
                 VLongWritable.class.getName());
         RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
                 partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
+        IConfigurationFactory configurationFactory = getConfigurationFactory();
         IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
         IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
+                getConfigurationFactory(), vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(),
+                vertexClass.getName());
 
         IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
                 spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
@@ -332,32 +305,12 @@
         setLocationConstraint(spec, join);
 
         /**
-         * construct local sort operator
+         * construct group-by operator pipeline
          */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, localSort);
-
-        /**
-         * construct local pre-clustered group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf,
-                false, false);
-        ClusteredGroupOperatorDescriptor localGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
-        setLocationConstraint(spec, localGby);
-
-        /**
-         * construct global group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, true);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
+        Pair<IOperatorDescriptor, IOperatorDescriptor> groupOps = generateGroupingOperators(spec, iteration,
+                vertexIdClass);
+        IOperatorDescriptor groupStartOperator = groupOps.getLeft();
+        IOperatorDescriptor groupEndOperator = groupOps.getRight();
 
         /**
          * construct the materializing write operator
@@ -386,7 +339,7 @@
          * final aggregate write operator
          */
         IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
+                getConfigurationFactory(), partialAggregateValueClassNames);
         FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
                 configurationFactory, aggRdFactory, jobId);
         PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
@@ -426,7 +379,7 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, groupStartOperator, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
                 terminateWriter, 0);
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
@@ -440,10 +393,7 @@
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
         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,
-                nkmFactory), localGby, 0, globalGby, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), groupEndOperator, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
 
@@ -463,4 +413,4 @@
         return cleanups;
     }
 
-}
\ No newline at end of file
+}
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
deleted file mode 100644
index a72777b..0000000
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ /dev/null
@@ -1,443 +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.core.jobgen;
-
-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;
-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.ITuplePartitionComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-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;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.pregelix.api.graph.MsgList;
-import edu.uci.ics.pregelix.api.job.PregelixJob;
-import edu.uci.ics.pregelix.api.util.BspUtils;
-import edu.uci.ics.pregelix.core.data.TypeTraits;
-import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
-import edu.uci.ics.pregelix.core.optimizer.IOptimizer;
-import edu.uci.ics.pregelix.core.util.DataflowUtils;
-import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
-import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.FinalAggregateOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.MaterializingReadOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.group.ClusteredGroupOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.group.IClusteredAggregatorDescriptorFactory;
-import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
-import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
-import edu.uci.ics.pregelix.runtime.function.StartComputeUpdateFunctionFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.MergePartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.MsgListNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.PostSuperStepRuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-
-public class JobGenOuterJoinSingleSort extends JobGen {
-
-    public JobGenOuterJoinSingleSort(PregelixJob job, IOptimizer optimizer) {
-        super(job, optimizer);
-    }
-
-    public JobGenOuterJoinSingleSort(PregelixJob job, String jobId, IOptimizer optimizer) {
-        super(job, jobId, optimizer);
-    }
-
-    @Override
-    protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
-        Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
-        Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
-        Class<? extends Writable> messageValueClass = BspUtils.getMessageValueClass(conf);
-        String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf);
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
-        JobSpecification spec = new JobSpecification(frameSize);
-
-        /**
-         * construct empty tuple operator
-         */
-        EmptyTupleSourceOperatorDescriptor emptyTupleSource = new EmptyTupleSourceOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptyTupleSource);
-
-        /** construct runtime hook */
-        RuntimeHookOperatorDescriptor preSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PreSuperStepRuntimeHookFactory(jobId, confFactory));
-        setLocationConstraint(spec, preSuperStep);
-
-        /**
-         * construct btree search operator
-         */
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
-                vertexIdClass.getName(), vertexClass.getName());
-        IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
-        IFileSplitProvider fileSplitProvider = getFileSplitProvider(jobId, PRIMARY_INDEX);
-
-        ITypeTraits[] typeTraits = new ITypeTraits[2];
-        typeTraits[0] = new TypeTraits(false);
-        typeTraits[1] = new TypeTraits(false);
-
-        /**
-         * construct compute operator
-         */
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
-        IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
-        IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
-                vertexIdClass.getName(), messageValueClass.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
-
-        TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
-                recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
-                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
-        setLocationConstraint(spec, scanner);
-
-        /**
-         * construct global sort operator
-         */
-        int[] keyFields = new int[] { 0 };
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, globalSort);
-
-        /**
-         * construct global group-by operator
-         */
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                MsgList.class.getName());
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, false);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
-
-        /**
-         * construct the materializing write operator
-         */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
-                jobId, iteration);
-        setLocationConstraint(spec, materialize);
-
-        RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PostSuperStepRuntimeHookFactory(jobId));
-        setLocationConstraint(spec, postSuperStep);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink2 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink2);
-
-        /**
-         * termination state write operator
-         */
-        TerminationStateWriterOperatorDescriptor terminateWriter = new TerminationStateWriterOperatorDescriptor(spec,
-                configurationFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, terminateWriter, 1);
-
-        /**
-         * final aggregate write operator
-         */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
-        FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
-                configurationFactory, aggRdFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
-
-        /**
-         * add the insert operator to insert vertexes
-         */
-        int[] fieldPermutation = new int[] { 0, 1 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, insertOp);
-
-        /**
-         * add the delete operator to delete vertexes
-         */
-        int[] fieldPermutationDelete = new int[] { 0 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, deleteOp);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink3);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink4);
-
-        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
-        /** connect all operators **/
-        spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 0, globalSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
-                terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
-                finalAggregator, 0);
-
-        /**
-         * connect the insert/delete operator
-         */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 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);
-
-        spec.addRoot(terminateWriter);
-        spec.addRoot(finalAggregator);
-        spec.addRoot(emptySink2);
-        spec.addRoot(emptySink3);
-        spec.addRoot(emptySink4);
-
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
-        spec.setFrameSize(frameSize);
-        return spec;
-    }
-
-    @Override
-    protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
-        Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
-        Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
-        Class<? extends Writable> messageValueClass = BspUtils.getMessageValueClass(conf);
-        String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf);
-        JobSpecification spec = new JobSpecification(frameSize);
-
-        /**
-         * source aggregate
-         */
-        int[] keyFields = new int[] { 0 };
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
-                vertexIdClass.getName(), messageValueClass.getName());
-        IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                MsgList.class.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
-
-        /**
-         * construct empty tuple operator
-         */
-        EmptyTupleSourceOperatorDescriptor emptyTupleSource = new EmptyTupleSourceOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptyTupleSource);
-
-        /**
-         * construct pre-superstep hook
-         */
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
-        RuntimeHookOperatorDescriptor preSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PreSuperStepRuntimeHookFactory(jobId, confFactory));
-        setLocationConstraint(spec, preSuperStep);
-
-        /**
-         * construct the materializing write operator
-         */
-        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
-                true, jobId, iteration);
-        setLocationConstraint(spec, materializeRead);
-
-        /**
-         * construct index join function update operator
-         */
-        IFileSplitProvider fileSplitProvider = getFileSplitProvider(jobId, PRIMARY_INDEX);
-        ITypeTraits[] typeTraits = new ITypeTraits[2];
-        typeTraits[0] = new TypeTraits(false);
-        typeTraits[1] = new TypeTraits(false);
-        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
-        nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
-        nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
-
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
-        IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
-        IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
-
-        IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
-                getIndexDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
-                new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
-        setLocationConstraint(spec, join);
-
-        /**
-         * construct global sort operator
-         */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, globalSort);
-
-        /**
-         * construct global group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, false);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
-
-        /**
-         * construct the materializing write operator
-         */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
-                jobId, iteration);
-        setLocationConstraint(spec, materialize);
-
-        /** construct runtime hook */
-        RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PostSuperStepRuntimeHookFactory(jobId));
-        setLocationConstraint(spec, postSuperStep);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink);
-
-        /**
-         * termination state write operator
-         */
-        TerminationStateWriterOperatorDescriptor terminateWriter = new TerminationStateWriterOperatorDescriptor(spec,
-                configurationFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, terminateWriter, 1);
-
-        /**
-         * final aggregate write operator
-         */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
-        FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
-                configurationFactory, aggRdFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
-
-        int[] fieldPermutation = new int[] { 0, 1 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, insertOp);
-
-        /**
-         * add the delete operator to delete vertexes
-         */
-        int[] fieldPermutationDelete = new int[] { 0 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, deleteOp);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink3);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink4);
-
-        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
-
-        /** connect all operators **/
-        spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 0, globalSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
-                terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
-                finalAggregator, 0);
-        /**
-         * connect the insert/delete operator
-         */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 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);
-
-        spec.addRoot(terminateWriter);
-        spec.addRoot(finalAggregator);
-        spec.addRoot(emptySink);
-
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
-        spec.setFrameSize(frameSize);
-        return spec;
-    }
-
-    @Override
-    public JobSpecification[] generateCleanup() throws HyracksException {
-        JobSpecification[] cleanups = new JobSpecification[1];
-        cleanups[0] = this.dropIndex(PRIMARY_INDEX);
-        return cleanups;
-    }
-
-}
\ No newline at end of file
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
deleted file mode 100644
index e28b06b..0000000
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ /dev/null
@@ -1,476 +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.core.jobgen;
-
-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;
-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.ITuplePartitionComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-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;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.pregelix.api.graph.MsgList;
-import edu.uci.ics.pregelix.api.job.PregelixJob;
-import edu.uci.ics.pregelix.api.util.BspUtils;
-import edu.uci.ics.pregelix.core.data.TypeTraits;
-import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
-import edu.uci.ics.pregelix.core.optimizer.IOptimizer;
-import edu.uci.ics.pregelix.core.util.DataflowUtils;
-import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
-import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.FinalAggregateOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.MaterializingReadOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
-import edu.uci.ics.pregelix.dataflow.group.ClusteredGroupOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.group.IClusteredAggregatorDescriptorFactory;
-import edu.uci.ics.pregelix.dataflow.std.IndexNestedLoopJoinFunctionUpdateOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.std.TreeSearchFunctionUpdateOperatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
-import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.function.ComputeUpdateFunctionFactory;
-import edu.uci.ics.pregelix.runtime.function.StartComputeUpdateFunctionFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.MergePartitionComputerFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.MsgListNullWriterFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.PostSuperStepRuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.PreSuperStepRuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.RuntimeHookFactory;
-import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNullWriterFactory;
-
-public class JobGenOuterJoinSort extends JobGen {
-
-    public JobGenOuterJoinSort(PregelixJob job, IOptimizer optimizer) {
-        super(job, optimizer);
-    }
-
-    @Override
-    protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
-        Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
-        Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
-        Class<? extends Writable> messageValueClass = BspUtils.getMessageValueClass(conf);
-        String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf);
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
-        JobSpecification spec = new JobSpecification(frameSize);
-
-        /**
-         * construct empty tuple operator
-         */
-        EmptyTupleSourceOperatorDescriptor emptyTupleSource = new EmptyTupleSourceOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptyTupleSource);
-
-        /** construct runtime hook */
-        RuntimeHookOperatorDescriptor preSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PreSuperStepRuntimeHookFactory(jobId, confFactory));
-        setLocationConstraint(spec, preSuperStep);
-
-        /**
-         * construct btree search function update operator
-         */
-        RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
-                vertexIdClass.getName(), vertexClass.getName());
-        IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
-        IFileSplitProvider fileSplitProvider = getFileSplitProvider(jobId, PRIMARY_INDEX);
-
-        ITypeTraits[] typeTraits = new ITypeTraits[2];
-        typeTraits[0] = new TypeTraits(false);
-        typeTraits[1] = new TypeTraits(false);
-
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
-        IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
-        IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), vertexClass.getName());
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
-                vertexIdClass.getName(), messageValueClass.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
-
-        TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec,
-                recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, JobGenUtil.getForwardScan(iteration), null, null, true, true,
-                getIndexDataflowHelperFactory(), inputRdFactory, 5, new StartComputeUpdateFunctionFactory(confFactory),
-                preHookFactory, null, rdUnnestedMessage, rdDummy, rdPartialAggregate, rdInsert, rdDelete);
-        setLocationConstraint(spec, scanner);
-
-        /**
-         * construct local sort operator
-         */
-        int[] keyFields = new int[] { 0 };
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, localSort);
-
-        /**
-         * construct local pre-clustered group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf,
-                false, false);
-        ClusteredGroupOperatorDescriptor localGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
-        setLocationConstraint(spec, localGby);
-
-        /**
-         * construct global sort operator
-         */
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, globalSort);
-
-        /**
-         * construct global group-by operator
-         */
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                MsgList.class.getName());
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, true);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
-
-        /**
-         * construct the materializing write operator
-         */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
-                jobId, iteration);
-        setLocationConstraint(spec, materialize);
-
-        RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PostSuperStepRuntimeHookFactory(jobId));
-        setLocationConstraint(spec, postSuperStep);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink2 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink2);
-
-        /**
-         * termination state write operator
-         */
-        TerminationStateWriterOperatorDescriptor terminateWriter = new TerminationStateWriterOperatorDescriptor(spec,
-                configurationFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, terminateWriter, 1);
-        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-
-        /**
-         * final aggregate write operator
-         */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
-        FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
-                configurationFactory, aggRdFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
-
-        /**
-         * add the insert operator to insert vertexes
-         */
-        int[] fieldPermutation = new int[] { 0, 1 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, insertOp);
-
-        /**
-         * add the delete operator to delete vertexes
-         */
-        int[] fieldPermutationDelete = new int[] { 0 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, deleteOp);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink3);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink4);
-
-        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
-        /** connect all operators **/
-        spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, scanner, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, localSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 1,
-                terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), scanner, 2,
-                finalAggregator, 0);
-        /**
-         * connect the insert/delete operator
-         */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 3, insertOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 4, deleteOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalSort, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 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);
-
-        spec.addRoot(terminateWriter);
-        spec.addRoot(finalAggregator);
-        spec.addRoot(emptySink2);
-        spec.addRoot(emptySink3);
-        spec.addRoot(emptySink4);
-
-        spec.setFrameSize(frameSize);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
-        return spec;
-    }
-
-    @Override
-    protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
-        Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
-        Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
-        Class<? extends Writable> messageValueClass = BspUtils.getMessageValueClass(conf);
-        String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf);
-        JobSpecification spec = new JobSpecification(frameSize);
-
-        /**
-         * source aggregate
-         */
-        int[] keyFields = new int[] { 0 };
-        RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf,
-                vertexIdClass.getName(), messageValueClass.getName());
-        IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);
-        RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                MsgList.class.getName());
-        RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(),
-                vertexClass.getName());
-        RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName());
-
-        /**
-         * construct empty tuple operator
-         */
-        EmptyTupleSourceOperatorDescriptor emptyTupleSource = new EmptyTupleSourceOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptyTupleSource);
-
-        /**
-         * construct pre-superstep hook
-         */
-        IConfigurationFactory confFactory = new ConfigurationFactory(conf);
-        RuntimeHookOperatorDescriptor preSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PreSuperStepRuntimeHookFactory(jobId, confFactory));
-        setLocationConstraint(spec, preSuperStep);
-
-        /**
-         * construct the materializing write operator
-         */
-        MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
-                true, jobId, iteration);
-        setLocationConstraint(spec, materializeRead);
-
-        /**
-         * construct index join function update operator
-         */
-        IFileSplitProvider fileSplitProvider = getFileSplitProvider(jobId, PRIMARY_INDEX);
-        ITypeTraits[] typeTraits = new ITypeTraits[2];
-        typeTraits[0] = new TypeTraits(false);
-        typeTraits[1] = new TypeTraits(false);
-        INullWriterFactory[] nullWriterFactories = new INullWriterFactory[2];
-        nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE;
-        nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE;
-
-        RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                VLongWritable.class.getName());
-        RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf,
-                partialAggregateValueClassNames);
-        IConfigurationFactory configurationFactory = new ConfigurationFactory(conf);
-        IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory);
-        IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName());
-
-        IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor(
-                spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories,
-                JobGenUtil.getForwardScan(iteration), keyFields, keyFields, true, true,
-                getIndexDataflowHelperFactory(), true, nullWriterFactories, inputRdFactory, 5,
-                new ComputeUpdateFunctionFactory(confFactory), preHookFactory, null, rdUnnestedMessage, rdDummy,
-                rdPartialAggregate, rdInsert, rdDelete);
-        setLocationConstraint(spec, join);
-
-        /**
-         * construct local sort operator
-         */
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
-        IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
-        sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
-                .getClass());
-        ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, localSort);
-
-        /**
-         * construct local pre-clustered group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactory = DataflowUtils.getAccumulatingAggregatorFactory(conf,
-                false, false);
-        ClusteredGroupOperatorDescriptor localGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactory, rdUnnestedMessage);
-        setLocationConstraint(spec, localGby);
-
-        /**
-         * construct global sort operator
-         */
-        ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
-        setLocationConstraint(spec, globalSort);
-
-        /**
-         * construct global group-by operator
-         */
-        IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory(
-                conf, true, true);
-        ClusteredGroupOperatorDescriptor globalGby = new ClusteredGroupOperatorDescriptor(spec, keyFields,
-                sortCmpFactories, aggregatorFactoryFinal, rdFinal);
-        setLocationConstraint(spec, globalGby);
-
-        /**
-         * construct the materializing write operator
-         */
-        MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal,
-                jobId, iteration);
-        setLocationConstraint(spec, materialize);
-
-        /** construct runtime hook */
-        RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
-                new PostSuperStepRuntimeHookFactory(jobId));
-        setLocationConstraint(spec, postSuperStep);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink);
-
-        /**
-         * termination state write operator
-         */
-        TerminationStateWriterOperatorDescriptor terminateWriter = new TerminationStateWriterOperatorDescriptor(spec,
-                configurationFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, terminateWriter, 1);
-
-        /**
-         * final aggregate write operator
-         */
-        IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
-                conf, partialAggregateValueClassNames);
-        FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec,
-                configurationFactory, aggRdFactory, jobId);
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1);
-
-        /**
-         * add the insert operator to insert vertexes
-         */
-        int[] fieldPermutation = new int[] { 0, 1 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor insertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdInsert, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutation, IndexOperation.INSERT, getIndexDataflowHelperFactory(),
-                null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, insertOp);
-
-        /**
-         * add the delete operator to delete vertexes
-         */
-        int[] fieldPermutationDelete = new int[] { 0 };
-        TreeIndexInsertUpdateDeleteOperatorDescriptor deleteOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, rdDelete, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, null, fieldPermutationDelete, IndexOperation.DELETE,
-                getIndexDataflowHelperFactory(), null, NoOpOperationCallbackFactory.INSTANCE);
-        setLocationConstraint(spec, deleteOp);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink3 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink3);
-
-        /** construct empty sink operator */
-        EmptySinkOperatorDescriptor emptySink4 = new EmptySinkOperatorDescriptor(spec);
-        setLocationConstraint(spec, emptySink4);
-
-        ITuplePartitionComputerFactory unifyingPartitionComputerFactory = new MergePartitionComputerFactory();
-        ITuplePartitionComputerFactory partionFactory = getVertexPartitionComputerFactory();
-
-        /** connect all operators **/
-        spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, preSuperStep, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), preSuperStep, 0, materializeRead, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, join, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, localSort, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 1,
-                terminateWriter, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, unifyingPartitionComputerFactory), join, 2,
-                finalAggregator, 0);
-        /**
-         * connect the insert/delete operator
-         */
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 3, insertOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), insertOp, 0, emptySink3, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 4, deleteOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), localGby, 0, globalSort, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), globalSort, 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);
-
-        spec.addRoot(terminateWriter);
-        spec.addRoot(finalAggregator);
-        spec.addRoot(emptySink);
-        spec.addRoot(emptySink3);
-        spec.addRoot(emptySink4);
-
-        spec.setFrameSize(frameSize);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy(spec));
-        return spec;
-    }
-
-    @Override
-    public JobSpecification[] generateCleanup() throws HyracksException {
-        JobSpecification[] cleanups = new JobSpecification[1];
-        cleanups[0] = this.dropIndex(PRIMARY_INDEX);
-        return cleanups;
-    }
-
-}
\ No newline at end of file
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/optimizer/DynamicOptimizer.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/optimizer/DynamicOptimizer.java
index 064ca42..2dbaf88 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/optimizer/DynamicOptimizer.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/optimizer/DynamicOptimizer.java
@@ -16,9 +16,9 @@
 package edu.uci.ics.pregelix.core.optimizer;
 
 import java.io.File;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.TreeMap;
 
 import org.apache.hadoop.io.IntWritable;
 
@@ -37,7 +37,7 @@
 public class DynamicOptimizer implements IOptimizer {
 
     private IClusterCounterContext counterContext;
-    private Map<String, IntWritable> machineToDegreeOfParallelism = new HashMap<String, IntWritable>();
+    private Map<String, IntWritable> machineToDegreeOfParallelism = new TreeMap<String, IntWritable>();
     private int dop = 0;
 
     public DynamicOptimizer(IClusterCounterContext counterContext) {
@@ -47,7 +47,9 @@
     @Override
     public JobGen optimize(JobGen jobGen, int iteration) {
         try {
-            initializeLoadPerMachine();
+            if (iteration == 0) {
+                initializeLoadPerMachine();
+            }
             return jobGen;
         } catch (Exception e) {
             throw new IllegalStateException(e);
@@ -61,7 +63,7 @@
             int index = 0;
             for (Entry<String, IntWritable> entry : machineToDegreeOfParallelism.entrySet()) {
                 String loc = entry.getKey();
-                IntWritable count = machineToDegreeOfParallelism.get(loc);
+                IntWritable count = entry.getValue();
                 for (int j = 0; j < count.get(); j++) {
                     constraints[index++] = loc;
                 }
@@ -79,7 +81,7 @@
         int splitIndex = 0;
         for (Entry<String, IntWritable> entry : machineToDegreeOfParallelism.entrySet()) {
             String ncName = entry.getKey();
-            IntWritable count = machineToDegreeOfParallelism.get(ncName);
+            IntWritable count = entry.getValue();
             for (int j = 0; j < count.get(); j++) {
                 //cycles stores, each machine has the number of stores = the number of cores
                 int storeCursor = j % stores.length;
@@ -108,7 +110,8 @@
         for (Entry<String, IntWritable> entry : machineToDegreeOfParallelism.entrySet()) {
             String loc = entry.getKey();
             //reserve one core for heartbeat
-            int load = (int) counterContext.getCounter(Counters.NUM_PROCESSOR, false).get() - 1;
+            int load = (int) counterContext.getCounter(Counters.NUM_PROCESSOR, false).get();
+            //load = load > 3 ? load - 2 : load;
             IntWritable count = machineToDegreeOfParallelism.get(loc);
             count.set(load);
             dop += load;
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
index c0173ed..448a80f 100644
--- 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
@@ -31,6 +31,9 @@
 
             @Override
             public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                if (b1 == b2 && s1 == s2) {
+                    return 0;
+                }
                 int commonLength = Math.min(l1, l2);
                 for (int i = 0; i < commonLength; i++) {
                     if (b1[s1 + i] != b2[s2 + i]) {
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableComparingBinaryComparatorFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableComparingBinaryComparatorFactory.java
index a85bf05..ae80e90 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableComparingBinaryComparatorFactory.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableComparingBinaryComparatorFactory.java
@@ -36,6 +36,9 @@
         return new IBinaryComparator() {
             @Override
             public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                if (b1 == b2 && s1 == s2) {
+                    return 0;
+                }
                 return instance.compare(b1, s1, l1, b2, s2, l2);
             }
         };
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 68e3ba7..714253e 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
@@ -20,7 +20,6 @@
 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;
@@ -30,9 +29,9 @@
     private String[] fieldClasses;
     private IConfigurationFactory confFactory;
 
-    public WritableRecordDescriptorFactory(Configuration conf, String... fieldClasses) {
+    public WritableRecordDescriptorFactory(IConfigurationFactory confFactory, String... fieldClasses) {
         this.fieldClasses = fieldClasses;
-        this.confFactory = new ConfigurationFactory(conf);
+        this.confFactory = confFactory;
     }
 
     @Override
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 3a2241b..d0a2b80 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
@@ -21,13 +21,17 @@
 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.HyracksException;
-import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableRecordDescriptorFactory;
-import edu.uci.ics.pregelix.dataflow.group.IClusteredAggregatorDescriptorFactory;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IAggregateFunctionFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
-import edu.uci.ics.pregelix.runtime.simpleagg.AccumulatingAggregatorFactory;
-import edu.uci.ics.pregelix.runtime.simpleagg.AggregationFunctionFactory;
+import edu.uci.ics.pregelix.dataflow.std.base.ISerializableAggregateFunctionFactory;
+import edu.uci.ics.pregelix.dataflow.std.group.IClusteredAggregatorDescriptorFactory;
+import edu.uci.ics.pregelix.runtime.agg.AccumulatingAggregatorFactory;
+import edu.uci.ics.pregelix.runtime.agg.AggregationFunctionFactory;
+import edu.uci.ics.pregelix.runtime.agg.SerializableAggregationFunctionFactory;
+import edu.uci.ics.pregelix.runtime.agg.SerializableAggregatorDescriptorFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.DatatypeHelper;
 
 public class DataflowUtils {
@@ -62,7 +66,7 @@
             int i = 0;
             for (String className : classNames)
                 serdes[i++] = DatatypeHelper.createSerializerDeserializer(
-                        (Class<? extends Writable>) loader.loadClass(className), conf);
+                        (Class<? extends Writable>) loader.loadClass(className), conf, null);
         } catch (ClassNotFoundException cnfe) {
             throw new HyracksException(cnfe);
         }
@@ -70,21 +74,29 @@
         return recordDescriptor;
     }
 
-    public static IRecordDescriptorFactory getWritableRecordDescriptorFactoryFromWritableClasses(Configuration conf,
-            String... classNames) throws HyracksException {
-        IRecordDescriptorFactory rdFactory = new WritableRecordDescriptorFactory(conf, classNames);
+    public static IRecordDescriptorFactory getWritableRecordDescriptorFactoryFromWritableClasses(
+            IConfigurationFactory confFactory, String... classNames) throws HyracksException {
+        IRecordDescriptorFactory rdFactory = new WritableRecordDescriptorFactory(confFactory, classNames);
         return rdFactory;
     }
 
-    public static IClusteredAggregatorDescriptorFactory getAccumulatingAggregatorFactory(Configuration conf,
-            boolean isFinal, boolean partialAggAsInput) {
-        IAggregateFunctionFactory aggFuncFactory = new AggregationFunctionFactory(new ConfigurationFactory(conf),
-                isFinal, partialAggAsInput);
+    public static IClusteredAggregatorDescriptorFactory getAccumulatingAggregatorFactory(
+            IConfigurationFactory confFactory, boolean isFinal, boolean partialAggAsInput) {
+        IAggregateFunctionFactory aggFuncFactory = new AggregationFunctionFactory(confFactory, isFinal,
+                partialAggAsInput);
         IClusteredAggregatorDescriptorFactory aggregatorFactory = new AccumulatingAggregatorFactory(
                 new IAggregateFunctionFactory[] { aggFuncFactory });
         return aggregatorFactory;
     }
 
+    public static IAggregatorDescriptorFactory getSerializableAggregatorFactory(IConfigurationFactory confFactory,
+            boolean isFinal, boolean partialAggAsInput) {
+        ISerializableAggregateFunctionFactory aggFuncFactory = new SerializableAggregationFunctionFactory(confFactory,
+                partialAggAsInput);
+        IAggregatorDescriptorFactory aggregatorFactory = new SerializableAggregatorDescriptorFactory(aggFuncFactory);
+        return aggregatorFactory;
+    }
+
     @SuppressWarnings("unchecked")
     public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(IHyracksTaskContext ctx, Configuration conf,
             String className1, String className2) throws HyracksException {
@@ -108,7 +120,7 @@
             int i = 0;
             for (String className : classNames) {
                 Class<? extends Writable> c = (Class<? extends Writable>) ctx.getJobletContext().loadClass(className);
-                serdes[i++] = DatatypeHelper.createSerializerDeserializer(c, conf);
+                serdes[i++] = DatatypeHelper.createSerializerDeserializer(c, conf, ctx);
                 //System.out.println("thread " + Thread.currentThread().getId() + " after creating serde " + c.getClassLoader());
             }
         } catch (Exception cnfe) {
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java
index a4c4501..c2a303c 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java
@@ -35,6 +35,13 @@
     public static boolean recoverable(Exception exception, Set<String> blackListNodes) {
         String message = exception.getMessage();
 
+        /**
+         * Don't know to recover or not, return true
+         */
+        if (message == null) {
+            return true;
+        }
+
         /***
          * check interrupted exception
          */
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 70de9ed..13a08b7 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 = 10;
+        ccConfig.maxHeartbeatLapsePeriods = 20;
 
         // cluster controller
         cc = new ClusterControllerService(ccConfig);
@@ -122,6 +122,7 @@
 
     public static void runJob(JobSpecification spec, String appName) throws Exception {
         spec.setFrameSize(FRAME_SIZE);
+        spec.setReportTaskDetails(false);
         JobId jobId = hcc.startJob(spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
         hcc.waitForCompletion(jobId);
     }
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 f599996..5478ed9 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
@@ -282,7 +282,7 @@
             typeTraits[i] = new TypeTraits(false);
         TreeIndexBulkLoadOperatorDescriptor writer = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, null,
-                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 0, false, new BTreeDataflowHelperFactory(),
+                fieldPermutation, DEFAULT_BTREE_FILL_FACTOR, false, 100000, false, new BTreeDataflowHelperFactory(),
                 NoOpOperationCallbackFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, NC1_ID, NC2_ID);
 
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializableAggregateFunction.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializableAggregateFunction.java
new file mode 100644
index 0000000..489135f
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializableAggregateFunction.java
@@ -0,0 +1,30 @@
+/*
+ * 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.std.base;
+
+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.IFrameTupleReference;
+
+public interface ISerializableAggregateFunction {
+    /** should be called each time a new aggregate value is computed */
+    public void init(IFrameTupleReference tuple, ArrayTupleBuilder state) throws HyracksDataException;
+
+    public void step(IFrameTupleReference tuple, IFrameTupleReference state) throws HyracksDataException;
+
+    public void finishPartial(IFrameTupleReference state, ArrayTupleBuilder output) throws HyracksDataException;
+
+    public void finishFinal(IFrameTupleReference state, ArrayTupleBuilder output) throws HyracksDataException;
+}
diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializableAggregateFunctionFactory.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializableAggregateFunctionFactory.java
new file mode 100644
index 0000000..3abbb3b
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializableAggregateFunctionFactory.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.dataflow.std.base;
+
+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.exceptions.HyracksException;
+
+public interface ISerializableAggregateFunctionFactory extends Serializable {
+    public ISerializableAggregateFunction createAggregateFunction(IHyracksTaskContext ctx, IFrameWriter writer)
+            throws HyracksException;
+}
\ No newline at end of file
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 b5a2927..3512a23 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
@@ -41,11 +41,11 @@
 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.IUpdateFunctionFactory;
-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;
+import edu.uci.ics.pregelix.dataflow.std.util.CopyUpdateUtil;
+import edu.uci.ics.pregelix.dataflow.std.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.std.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.std.util.StorageType;
+import edu.uci.ics.pregelix.dataflow.std.util.UpdateBuffer;
 
 public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
     private IndexDataflowHelper treeIndexOpHelper;
@@ -220,20 +220,18 @@
     @Override
     public void close() throws HyracksDataException {
         try {
-            try {
-                cursor.close();
-                //batch update
-                updateBuffer.updateIndex(indexAccessor);
-            } catch (Exception e) {
-                throw new HyracksDataException(e);
-            }
-
+            cursor.close();
+            //batch update
+            updateBuffer.updateIndex(indexAccessor);
+        } catch (Exception e) {
+            closeResource();
+            throw new HyracksDataException(e);
+        } finally {
+            treeIndexOpHelper.close();
             /**
              * close the update function
              */
             functionProxy.functionClose();
-        } finally {
-            treeIndexOpHelper.close();
         }
     }
 
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 2a7fede..18675a1 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
@@ -43,11 +43,11 @@
 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.IUpdateFunctionFactory;
-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;
+import edu.uci.ics.pregelix.dataflow.std.util.CopyUpdateUtil;
+import edu.uci.ics.pregelix.dataflow.std.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.std.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.std.util.StorageType;
+import edu.uci.ics.pregelix.dataflow.std.util.UpdateBuffer;
 
 public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable extends
         AbstractUnaryInputOperatorNodePushable {
@@ -318,7 +318,7 @@
         /**
          * function call
          */
-        functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb, cursor);
+        functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb, cursor, true);
 
         //doing clone update
         CopyUpdateUtil.copyUpdate(tempTupleReference, frameTuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor,
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 fe27029..aab647b 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
@@ -41,11 +41,11 @@
 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.IUpdateFunctionFactory;
-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;
+import edu.uci.ics.pregelix.dataflow.std.util.CopyUpdateUtil;
+import edu.uci.ics.pregelix.dataflow.std.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.std.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.std.util.StorageType;
+import edu.uci.ics.pregelix.dataflow.std.util.UpdateBuffer;
 
 public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
     private IndexDataflowHelper treeIndexOpHelper;
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 1c9fce6..89d5e3c 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
@@ -125,7 +125,7 @@
             }
 
         } catch (Exception e) {
-            treeIndexOpHelper.close();
+            closeResource();
             throw new HyracksDataException(e);
         }
     }
@@ -158,6 +158,7 @@
                 }
             }
         } catch (Exception e) {
+            closeResource();
             throw new HyracksDataException(e);
         }
     }
@@ -190,21 +191,27 @@
             if (appender.getTupleCount() > 0) {
                 FrameUtils.flushFrame(writeBuffer, writer);
             }
-            writer.close();
             try {
                 cursor.close();
             } catch (Exception e) {
                 throw new HyracksDataException(e);
             }
         } catch (Exception e) {
+            closeResource();
             throw new HyracksDataException(e);
-        } finally {
+        } finally{
             treeIndexOpHelper.close();
+            writer.close();
         }
     }
 
     @Override
     public void fail() throws HyracksDataException {
+        closeResource();
+        populateFailure();
+    }
+
+    private void closeResource() throws HyracksDataException {
         try {
             cursor.close();
         } catch (Exception e) {
@@ -212,6 +219,9 @@
         } finally {
             treeIndexOpHelper.close();
         }
+    }
+
+    private void populateFailure() throws HyracksDataException {
         writer.fail();
     }
 
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
index c985f64..0056e8f 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
@@ -65,7 +65,7 @@
         treeIndexOpHelper.open();
         try {
             index = (ITreeIndex) treeIndexOpHelper.getIndexInstance();
-            bulkLoader = index.createBulkLoader(fillFactor, false, 0, false);
+            bulkLoader = index.createBulkLoader(fillFactor, false, 100000, false);
         } catch (Exception e) {
             // cleanup in case of failure
             treeIndexOpHelper.close();
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 f955831..da7288a 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
@@ -42,11 +42,11 @@
 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.IUpdateFunctionFactory;
-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;
+import edu.uci.ics.pregelix.dataflow.std.util.CopyUpdateUtil;
+import edu.uci.ics.pregelix.dataflow.std.util.FunctionProxy;
+import edu.uci.ics.pregelix.dataflow.std.util.SearchKeyTupleReference;
+import edu.uci.ics.pregelix.dataflow.std.util.StorageType;
+import edu.uci.ics.pregelix.dataflow.std.util.UpdateBuffer;
 
 public class TreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
     protected IndexDataflowHelper treeIndexHelper;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/collectors/SortMergeFrameReader.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/collectors/SortMergeFrameReader.java
new file mode 100644
index 0000000..31f4182
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/collectors/SortMergeFrameReader.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.dataflow.std.collectors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+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.std.collectors.IPartitionBatchManager;
+import edu.uci.ics.pregelix.dataflow.std.sort.RunMergingFrameReader;
+
+public class SortMergeFrameReader implements IFrameReader {
+    private IHyracksTaskContext ctx;
+    private final int maxConcurrentMerges;
+    private final int nSenders;
+    private final int[] sortFields;
+
+    private final RecordDescriptor recordDescriptor;
+    private final IPartitionBatchManager pbm;
+
+    private RunMergingFrameReader merger;
+
+    public SortMergeFrameReader(IHyracksTaskContext ctx, int maxConcurrentMerges, int nSenders, int[] sortFields,
+            RecordDescriptor recordDescriptor, IPartitionBatchManager pbm) {
+        this.ctx = ctx;
+        this.maxConcurrentMerges = maxConcurrentMerges;
+        this.nSenders = nSenders;
+        this.sortFields = sortFields;
+        this.recordDescriptor = recordDescriptor;
+        this.pbm = pbm;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        if (maxConcurrentMerges >= nSenders) {
+            List<ByteBuffer> inFrames = new ArrayList<ByteBuffer>();
+            for (int i = 0; i < nSenders; ++i) {
+                inFrames.add(ByteBuffer.allocate(ctx.getFrameSize()));
+            }
+            List<IFrameReader> batch = new ArrayList<IFrameReader>();
+            pbm.getNextBatch(batch, nSenders);
+            merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames, sortFields,
+                    recordDescriptor);
+        } else {
+            // multi level merge.
+            throw new HyracksDataException("Not yet supported");
+        }
+        merger.open();
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        buffer.position(buffer.capacity());
+        buffer.limit(buffer.capacity());
+        return merger.nextFrame(buffer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        merger.close();
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
new file mode 100644
index 0000000..2d820bc
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.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.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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;
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionBatchManager;
+import edu.uci.ics.hyracks.dataflow.std.collectors.PartitionCollector;
+import edu.uci.ics.hyracks.dataflow.std.connectors.PartitionDataWriter;
+import edu.uci.ics.pregelix.dataflow.std.collectors.SortMergeFrameReader;
+
+public class MToNPartitioningMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ITuplePartitionComputerFactory tpcf;
+    private final int[] sortFields;
+
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITuplePartitionComputerFactory tpcf, int[] sortFields) {
+        this(spec, tpcf, sortFields, false);
+    }
+
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITuplePartitionComputerFactory tpcf, int[] sortFields, boolean stable) {
+        super(spec);
+        this.tpcf = tpcf;
+        this.sortFields = sortFields;
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
+                recordDesc, tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
+        IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
+                sortFields, recordDesc, pbm);
+        BitSet expectedPartitions = new BitSet();
+        expectedPartitions.set(0, nProducerPartitions);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupOperatorDescriptor.java
similarity index 97%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupOperatorDescriptor.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupOperatorDescriptor.java
index bb41953..ed1141e 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupOperatorDescriptor.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.group;
+package edu.uci.ics.pregelix.dataflow.std.group;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupOperatorNodePushable.java
similarity index 98%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupOperatorNodePushable.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupOperatorNodePushable.java
index a95a46e..a86f28d 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupOperatorNodePushable.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupOperatorNodePushable.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.group;
+package edu.uci.ics.pregelix.dataflow.std.group;
 
 import java.nio.ByteBuffer;
 
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupWriter.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupWriter.java
similarity index 72%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupWriter.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupWriter.java
index 4b4a1c3..605ae19 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/ClusteredGroupWriter.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/ClusteredGroupWriter.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.group;
+package edu.uci.ics.pregelix.dataflow.std.group;
 
 import java.nio.ByteBuffer;
 
@@ -21,12 +21,10 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 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;
 
 public class ClusteredGroupWriter implements IFrameWriter {
     private final int[] groupFields;
@@ -40,8 +38,6 @@
 
     private final ByteBuffer outFrame;
     private final FrameTupleAppender appender;
-    private final ArrayTupleBuilder tupleBuilder;
-
     private boolean first;
 
     public ClusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
@@ -56,11 +52,11 @@
         copyFrameAccessor.reset(copyFrame);
 
         outFrame = ctx.allocateFrame();
-        appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender = new FrameTupleAppender(ctx.getFrameSize(), outRecordDesc.getFields().length);
         appender.reset(outFrame, true);
 
-        tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
-        this.aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc, outRecordDesc, groupFields, groupFields, writer, outFrame, appender);
+        this.aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc, outRecordDesc, groupFields,
+                groupFields, writer, outFrame, appender);
         this.aggregateState = aggregator.createAggregateStates();
     }
 
@@ -76,15 +72,8 @@
         int nTuples = inFrameAccessor.getTupleCount();
         for (int i = 0; i < nTuples; ++i) {
             if (first) {
-
-                tupleBuilder.reset();
-                for (int j = 0; j < groupFields.length; j++) {
-                    tupleBuilder.addField(inFrameAccessor, i, groupFields[j]);
-                }
-                aggregator.init(tupleBuilder, inFrameAccessor, i, aggregateState);
-
+                aggregator.init(inFrameAccessor, i, aggregateState);
                 first = false;
-
             } else {
                 if (i == 0) {
                     switchGroupIfRequired(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1, inFrameAccessor, i);
@@ -101,32 +90,20 @@
             FrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException {
         if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor, currTupleIndex)) {
             writeOutput(prevTupleAccessor, prevTupleIndex);
-
-            tupleBuilder.reset();
-            for (int j = 0; j < groupFields.length; j++) {
-                tupleBuilder.addField(currTupleAccessor, currTupleIndex, groupFields[j]);
-            }
-            aggregator.init(tupleBuilder, currTupleAccessor, currTupleIndex, aggregateState);
+            aggregator.init(currTupleAccessor, currTupleIndex, aggregateState);
         } else {
-            aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0, aggregateState);
+            aggregator.aggregate(currTupleAccessor, currTupleIndex, aggregateState);
         }
     }
 
     private void writeOutput(final FrameTupleAccessor lastTupleAccessor, int lastTupleIndex)
             throws HyracksDataException {
-        tupleBuilder.reset();
-        for (int j = 0; j < groupFields.length; j++) {
-            tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, groupFields[j]);
-        }
-        aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor, lastTupleIndex, aggregateState);
-        if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                tupleBuilder.getSize())) {
+        if (!aggregator.outputFinalResult(lastTupleAccessor, lastTupleIndex, aggregateState, appender)) {
             FrameUtils.flushFrame(outFrame, writer);
             appender.reset(outFrame, true);
-            if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                    tupleBuilder.getSize())) {
-                throw new HyracksDataException("The output of size " + tupleBuilder.getSize()
-                        + " cannot be fit into a frame of size " + outFrame.array().length);
+            if (!aggregator.outputFinalResult(lastTupleAccessor, lastTupleIndex, aggregateState, appender)) {
+                throw new HyracksDataException("The output of size " + " cannot be fit into a frame of size "
+                        + outFrame.array().length);
             }
         }
 
@@ -154,9 +131,11 @@
     @Override
     public void close() throws HyracksDataException {
         if (!first) {
-            writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1);
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(outFrame, writer);
+            if (copyFrameAccessor.getTupleCount() > 0) {
+                writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1);
+                if (appender.getTupleCount() > 0) {
+                    FrameUtils.flushFrame(outFrame, writer);
+                }
             }
         }
         aggregateState.close();
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/IAggregatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/IAggregatorDescriptor.java
new file mode 100644
index 0000000..a46ae65
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/IAggregatorDescriptor.java
@@ -0,0 +1,105 @@
+/*
+ * 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.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.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+
+public interface IAggregatorDescriptor {
+
+    /**
+     * Create an aggregate state
+     * 
+     * @return
+     */
+    public AggregateState createAggregateStates();
+
+    /**
+     * Initialize the state based on the input tuple.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param fieldOutput
+     *            The data output for the frame containing the state. This may
+     *            be null, if the state is maintained as a java object
+     * @param state
+     *            The state to be initialized.
+     * @throws HyracksDataException
+     */
+    public void init(IFrameTupleAccessor accessor, int tIndex, AggregateState state) throws HyracksDataException;
+
+    /**
+     * Reset the aggregator. The corresponding aggregate state should be reset
+     * too. Note that here the frame is not an input argument, since it can be
+     * reset outside of the aggregator (simply reset the starting index of the
+     * buffer).
+     * 
+     * @param state
+     */
+    public void reset();
+
+    /**
+     * Aggregate the value. Aggregate state should be updated correspondingly.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param data
+     *            The buffer containing the state, if frame-based-state is used.
+     *            This means that it can be null if java-object-based-state is
+     *            used.
+     * @param offset
+     * @param state
+     *            The aggregate state.
+     * @throws HyracksDataException
+     */
+    public void aggregate(IFrameTupleAccessor accessor, int tIndex, AggregateState state) throws HyracksDataException;
+
+    /**
+     * Output the partial aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @return TODO
+     * @throws HyracksDataException
+     */
+    public boolean outputPartialResult(IFrameTupleAccessor accessor, int tIndex, AggregateState state,
+            FrameTupleAppender appender) throws HyracksDataException;
+
+    /**
+     * Output the final aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @return true if the group is already written; false--left for the group writer to write the grouped tuple
+     * @throws HyracksDataException
+     */
+    public boolean outputFinalResult(IFrameTupleAccessor accessor, int tIndex, AggregateState state,
+            FrameTupleAppender appender) throws HyracksDataException;
+
+    public void close();
+
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/IClusteredAggregatorDescriptorFactory.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/IClusteredAggregatorDescriptorFactory.java
similarity index 92%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/IClusteredAggregatorDescriptorFactory.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/IClusteredAggregatorDescriptorFactory.java
index 3256f08..b082cbb 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/group/IClusteredAggregatorDescriptorFactory.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/group/IClusteredAggregatorDescriptorFactory.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.group;
+package edu.uci.ics.pregelix.dataflow.std.group;
 
 import java.io.Serializable;
 import java.nio.ByteBuffer;
@@ -22,7 +22,6 @@
 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.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
 
 /**
  *
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ExternalSortRunGenerator.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ExternalSortRunGenerator.java
new file mode 100644
index 0000000..c1c41d4
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -0,0 +1,114 @@
+/*
+ * 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.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+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.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.pregelix.dataflow.std.group.ClusteredGroupWriter;
+import edu.uci.ics.pregelix.dataflow.std.group.IClusteredAggregatorDescriptorFactory;
+
+public class ExternalSortRunGenerator implements IFrameWriter {
+    private final IHyracksTaskContext ctx;
+    private final IFrameSorter frameSorter;
+    private final List<IFrameReader> runs;
+    private final int maxSortFrames;
+
+    private final int[] groupFields;
+    private final IBinaryComparator[] comparators;
+    private final IClusteredAggregatorDescriptorFactory aggregatorFactory;
+    private final RecordDescriptor inRecordDesc;
+    private final RecordDescriptor outRecordDesc;
+
+    public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor recordDesc,
+            int framesLimit, int[] groupFields, IBinaryComparator[] comparators,
+            IClusteredAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor outRecordDesc)
+            throws HyracksDataException {
+        this.ctx = ctx;
+        this.frameSorter = new FrameSorterQuickSort(ctx, sortFields, recordDesc);
+        this.runs = new LinkedList<IFrameReader>();
+        this.maxSortFrames = framesLimit - 1;
+
+        this.groupFields = groupFields;
+        this.comparators = comparators;
+        this.aggregatorFactory = aggregatorFactory;
+        this.inRecordDesc = recordDesc;
+        this.outRecordDesc = outRecordDesc;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        runs.clear();
+        frameSorter.reset();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (frameSorter.getFrameCount() >= maxSortFrames) {
+            flushFramesToRun();
+        }
+        frameSorter.insertFrame(buffer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (frameSorter.getFrameCount() > 0) {
+            if (runs.size() <= 0) {
+                frameSorter.sortFrames();
+            } else {
+                flushFramesToRun();
+            }
+        }
+    }
+
+    private void flushFramesToRun() throws HyracksDataException {
+        frameSorter.sortFrames();
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                ExternalSortRunGenerator.class.getSimpleName());
+        RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
+        ClusteredGroupWriter pgw = new ClusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory,
+                this.inRecordDesc, this.outRecordDesc, writer);
+        pgw.open();
+
+        try {
+            frameSorter.flushFrames(pgw);
+        } finally {
+            pgw.close();
+        }
+        frameSorter.reset();
+        runs.add(writer.createReader());
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+    }
+
+    public IFrameSorter getFrameSorter() {
+        return frameSorter;
+    }
+
+    public List<IFrameReader> getRuns() {
+        return runs;
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ExternalSortRunMerger.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ExternalSortRunMerger.java
new file mode 100644
index 0000000..ff73ced
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ExternalSortRunMerger.java
@@ -0,0 +1,159 @@
+/*
+ * 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.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+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.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+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.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.pregelix.dataflow.std.group.ClusteredGroupWriter;
+import edu.uci.ics.pregelix.dataflow.std.group.IClusteredAggregatorDescriptorFactory;
+
+/**
+ * Group-by aggregation is pushed into multi-pass merge of external sort.
+ * 
+ * @author yingyib
+ */
+public class ExternalSortRunMerger {
+
+    private final IHyracksTaskContext ctx;
+    private final List<IFrameReader> runs;
+    private final int[] sortFields;
+    private final RecordDescriptor inRecordDesc;
+    private final RecordDescriptor outRecordDesc;
+    private final int framesLimit;
+    private final IFrameWriter writer;
+    private List<ByteBuffer> inFrames;
+    private ByteBuffer outFrame;
+    private FrameTupleAppender outFrameAppender;
+
+    private IFrameSorter frameSorter; // Used in External sort, no replacement
+                                      // selection
+
+    private final int[] groupFields;
+    private final IBinaryComparator[] comparators;
+    private final IClusteredAggregatorDescriptorFactory aggregatorFactory;
+    private final IClusteredAggregatorDescriptorFactory partialAggregatorFactory;
+    private final boolean localSide;
+
+    // Constructor for external sort, no replacement selection
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+            int[] sortFields, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, int framesLimit,
+            IFrameWriter writer, int[] groupFields, IBinaryComparator[] comparators,
+            IClusteredAggregatorDescriptorFactory partialAggregatorFactory,
+            IClusteredAggregatorDescriptorFactory aggregatorFactory, boolean localSide) {
+        this.ctx = ctx;
+        this.frameSorter = frameSorter;
+        this.runs = new LinkedList<IFrameReader>(runs);
+        this.sortFields = sortFields;
+        this.inRecordDesc = inRecordDesc;
+        this.outRecordDesc = outRecordDesc;
+        this.framesLimit = framesLimit;
+        this.writer = writer;
+
+        this.groupFields = groupFields;
+        this.comparators = comparators;
+        this.aggregatorFactory = aggregatorFactory;
+        this.partialAggregatorFactory = partialAggregatorFactory;
+        this.localSide = localSide;
+    }
+
+    public void process() throws HyracksDataException {
+        ClusteredGroupWriter pgw = new ClusteredGroupWriter(ctx, groupFields, comparators,
+                localSide ? partialAggregatorFactory : aggregatorFactory, inRecordDesc, outRecordDesc, writer);
+        try {
+            if (runs.size() <= 0) {
+                pgw.open();
+                if (frameSorter != null && frameSorter.getFrameCount() > 0) {
+                    frameSorter.flushFrames(pgw);
+                }
+                /** recycle sort buffer */
+                frameSorter.close();
+            } else {
+                /** recycle sort buffer */
+                frameSorter.close();
+
+                inFrames = new ArrayList<ByteBuffer>();
+                outFrame = ctx.allocateFrame();
+                outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+                outFrameAppender.reset(outFrame, true);
+                for (int i = 0; i < framesLimit - 1; ++i) {
+                    inFrames.add(ctx.allocateFrame());
+                }
+                int maxMergeWidth = framesLimit - 1;
+                while (runs.size() > maxMergeWidth) {
+                    int generationSeparator = 0;
+                    while (generationSeparator < runs.size() && runs.size() > maxMergeWidth) {
+                        int mergeWidth = Math.min(Math.min(runs.size() - generationSeparator, maxMergeWidth),
+                                runs.size() - maxMergeWidth + 1);
+                        FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class
+                                .getSimpleName());
+                        IFrameWriter mergeResultWriter = new RunFileWriter(newRun, ctx.getIOManager());
+                        pgw = new ClusteredGroupWriter(ctx, groupFields, comparators, partialAggregatorFactory,
+                                inRecordDesc, inRecordDesc, mergeResultWriter);
+                        pgw.open();
+                        IFrameReader[] runCursors = new RunFileReader[mergeWidth];
+                        for (int i = 0; i < mergeWidth; i++) {
+                            runCursors[i] = runs.get(generationSeparator + i);
+                        }
+                        merge(pgw, runCursors);
+                        pgw.close();
+                        runs.subList(generationSeparator, mergeWidth + generationSeparator).clear();
+                        runs.add(generationSeparator++, ((RunFileWriter) mergeResultWriter).createReader());
+                    }
+                }
+                if (!runs.isEmpty()) {
+                    pgw = new ClusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory, inRecordDesc,
+                            inRecordDesc, writer);
+                    pgw.open();
+                    IFrameReader[] runCursors = new RunFileReader[runs.size()];
+                    for (int i = 0; i < runCursors.length; i++) {
+                        runCursors[i] = runs.get(i);
+                    }
+                    merge(pgw, runCursors);
+                }
+            }
+        } catch (Exception e) {
+            pgw.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            pgw.close();
+        }
+    }
+
+    private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
+        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields, inRecordDesc);
+        merger.open();
+        try {
+            while (merger.nextFrame(outFrame)) {
+                FrameUtils.flushFrame(outFrame, mergeResultWriter);
+            }
+        } finally {
+            merger.close();
+        }
+    }
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/FastSortOperatorDescriptor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/FastSortOperatorDescriptor.java
new file mode 100644
index 0000000..85bc149
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/FastSortOperatorDescriptor.java
@@ -0,0 +1,188 @@
+/*
+ * 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.std.sort;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+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.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.pregelix.dataflow.std.group.IClusteredAggregatorDescriptorFactory;
+
+public class FastSortOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static final int SORT_ACTIVITY_ID = 0;
+    private static final int MERGE_ACTIVITY_ID = 1;
+
+    private final int[] sortFields;
+    private final int framesLimit;
+
+    private final int[] groupFields;
+    private final IClusteredAggregatorDescriptorFactory aggregatorFactory;
+    private final IClusteredAggregatorDescriptorFactory partialAggregatorFactory;
+    private final RecordDescriptor combinedRecordDesc;
+    private final RecordDescriptor outputRecordDesc;
+    private final boolean localSide;
+
+    public FastSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            RecordDescriptor recordDescriptor, int[] groupFields,
+            IClusteredAggregatorDescriptorFactory partialAggregatorFactory,
+            IClusteredAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor combinedRecordDesc,
+            RecordDescriptor outRecordDesc, boolean localSide) {
+        super(spec, 1, 1);
+        this.framesLimit = framesLimit;
+        this.sortFields = sortFields;
+        if (framesLimit <= 1) {
+            throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
+        }
+        this.recordDescriptors[0] = recordDescriptor;
+
+        this.groupFields = groupFields;
+        this.aggregatorFactory = aggregatorFactory;
+        this.partialAggregatorFactory = partialAggregatorFactory;
+        this.combinedRecordDesc = combinedRecordDesc;
+        this.outputRecordDesc = outRecordDesc;
+        this.localSide = localSide;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
+        MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
+
+        builder.addActivity(this, sa);
+        builder.addSourceEdge(0, sa, 0);
+
+        builder.addActivity(this, ma);
+        builder.addTargetEdge(0, ma, 0);
+
+        builder.addBlockingEdge(sa, ma);
+    }
+
+    public static class SortTaskState extends AbstractStateObject {
+        private List<IFrameReader> runs;
+        private IFrameSorter frameSorter;
+
+        public SortTaskState() {
+        }
+
+        private SortTaskState(JobId jobId, TaskId taskId) {
+            super(jobId, taskId);
+        }
+
+        @Override
+        public void toBytes(DataOutput out) throws IOException {
+
+        }
+
+        @Override
+        public void fromBytes(DataInput in) throws IOException {
+
+        }
+    }
+
+    private class SortActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public SortActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
+                private ExternalSortRunGenerator runGen;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    runGen = new ExternalSortRunGenerator(ctx, sortFields, recordDescriptors[0], framesLimit,
+                            groupFields, new IBinaryComparator[] { new RawBinaryComparator() },
+                            partialAggregatorFactory, combinedRecordDesc);
+                    runGen.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    runGen.nextFrame(buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(
+                            getActivityId(), partition));
+                    runGen.close();
+                    state.runs = runGen.getRuns();
+                    state.frameSorter = runGen.getFrameSorter();
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    runGen.fail();
+                }
+            };
+            return op;
+        }
+    }
+
+    private class MergeActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public MergeActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
+                @Override
+                public void initialize() throws HyracksDataException {
+                    SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
+                            SORT_ACTIVITY_ID), partition));
+                    List<IFrameReader> runs = state.runs;
+                    IFrameSorter frameSorter = state.frameSorter;
+                    int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
+                    ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
+                            combinedRecordDesc, outputRecordDesc, necessaryFrames, writer, groupFields,
+                            new IBinaryComparator[] { new RawBinaryComparator() }, partialAggregatorFactory,
+                            aggregatorFactory, localSide);
+                    merger.process();
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/FrameSorterQuickSort.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/FrameSorterQuickSort.java
new file mode 100644
index 0000000..d50e708
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/FrameSorterQuickSort.java
@@ -0,0 +1,250 @@
+/*
+ * 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.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+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.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 FrameSorterQuickSort implements IFrameSorter {
+    private final IHyracksTaskContext ctx;
+    private final int[] sortFields;
+    private final List<ByteBuffer> buffers;
+
+    private final FrameTupleAccessor fta1;
+    private final FrameTupleAccessor fta2;
+
+    private final FrameTupleAppender appender;
+
+    private final ByteBuffer outFrame;
+
+    private int dataFrameCount;
+    private int[] tPointers;
+    private int tupleCount;
+
+    private final RawBinaryComparator[] comparators = new RawBinaryComparator[] { new RawBinaryComparator() };
+    private final RawNormalizedKeyComputer nkc = new RawNormalizedKeyComputer();
+
+    public FrameSorterQuickSort(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor recordDescriptor)
+            throws HyracksDataException {
+        this.ctx = ctx;
+        this.sortFields = sortFields;
+        buffers = new ArrayList<ByteBuffer>();
+        fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        appender = new FrameTupleAppender(ctx.getFrameSize());
+        outFrame = ctx.allocateFrame();
+
+        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()) {
+            copyFrame = ctx.allocateFrame();
+            buffers.add(copyFrame);
+        } else {
+            copyFrame = buffers.get(dataFrameCount);
+        }
+        FrameUtils.copy(buffer, copyFrame);
+        ++dataFrameCount;
+    }
+
+    @Override
+    public void sortFrames() {
+        int nBuffers = dataFrameCount;
+        tupleCount = 0;
+        for (int i = 0; i < nBuffers; ++i) {
+            fta1.reset(buffers.get(i));
+            tupleCount += fta1.getTupleCount();
+        }
+        int sfIdx = sortFields[0];
+        tPointers = tPointers == null || tPointers.length < tupleCount * 4 ? new int[tupleCount * 4] : tPointers;
+        int ptr = 0;
+        for (int i = 0; i < nBuffers; ++i) {
+            fta1.reset(buffers.get(i));
+            int tCount = fta1.getTupleCount();
+            byte[] array = fta1.getBuffer().array();
+            for (int j = 0; j < tCount; ++j) {
+                int tStart = fta1.getTupleStartOffset(j);
+                int tEnd = fta1.getTupleEndOffset(j);
+                tPointers[ptr * 4] = i << 16;
+                tPointers[ptr * 4 + 1] = tStart;
+                tPointers[ptr * 4 + 2] = tEnd;
+                int f0StartRel = fta1.getFieldStartOffset(j, sfIdx);
+                int f0EndRel = fta1.getFieldEndOffset(j, sfIdx);
+                int f0Start = f0StartRel + tStart + fta1.getFieldSlotsLength();
+                tPointers[ptr * 4 + 3] = nkc == null ? 0 : nkc.normalize(array, f0Start, f0EndRel - f0StartRel);
+                tPointers[ptr * 4] |= nkc == null ? 0 : (nkc.normalize2(array, f0Start, f0EndRel - f0StartRel) & 0xff);
+                ++ptr;
+            }
+        }
+        if (tupleCount > 0) {
+            sort(tPointers, 0, tupleCount);
+        }
+    }
+
+    @Override
+    public void flushFrames(IFrameWriter writer) throws HyracksDataException {
+        appender.reset(outFrame, true);
+        for (int ptr = 0; ptr < tupleCount; ++ptr) {
+            int i = tPointers[ptr * 4] >>> 16;
+            int tStart = tPointers[ptr * 4 + 1];
+            int tEnd = tPointers[ptr * 4 + 2];
+            ByteBuffer buffer = buffers.get(i);
+            fta1.reset(buffer);
+            if (!appender.append(fta1, tStart, tEnd)) {
+                FrameUtils.flushFrame(outFrame, writer);
+                appender.reset(outFrame, true);
+                if (!appender.append(fta1, tStart, tEnd)) {
+                    throw new HyracksDataException("Record size (" + (tEnd - tStart) + ") larger than frame size ("
+                            + appender.getBuffer().capacity() + ")");
+                }
+            }
+        }
+        if (appender.getTupleCount() > 0) {
+            FrameUtils.flushFrame(outFrame, writer);
+        }
+    }
+
+    private void sort(int[] tPointers, int offset, int length) {
+        int m = offset + (length >> 1);
+        int mi = tPointers[m * 4] >>> 16;
+        int mu = tPointers[m * 4] & 0xff;
+        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, mu);
+                if (cmp > 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, a++, b);
+                }
+                ++b;
+            }
+            while (c >= b) {
+                int cmp = compare(tPointers, c, mi, mj, mv, mu);
+                if (cmp < 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, c, d--);
+                }
+                --c;
+            }
+            if (b > c)
+                break;
+            swap(tPointers, b++, c--);
+        }
+
+        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 ((s = d - c) > 1) {
+            sort(tPointers, n - s, s);
+        }
+    }
+
+    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 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 tp2u) {
+        int v1 = tPointers[tp1 * 4 + 3];
+        if (v1 != tp2v) {
+            return v1 < tp2v ? -1 : 1;
+        }
+        int u1 = tPointers[tp1 * 4] & 0xff;
+        if (u1 != tp2u) {
+            return u1 < tp2u ? -1 : 1;
+        }
+        int i1 = tPointers[tp1 * 4] >>> 16;
+        int j1 = tPointers[tp1 * 4 + 1];
+        int i2 = tp2i;
+        int j2 = tp2j;
+        ByteBuffer buf1 = buffers.get(i1);
+        ByteBuffer buf2 = buffers.get(i2);
+        byte[] b1 = buf1.array();
+        byte[] b2 = buf2.array();
+        fta1.reset(buf1);
+        fta2.reset(buf2);
+        for (int f = 0; f < comparators.length; ++f) {
+            int fIdx = sortFields[f];
+            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b1, j1 + (fIdx - 1) * 4);
+            int f1End = IntSerDeUtils.getInt(b1, j1 + fIdx * 4);
+            int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
+            int l1 = f1End - f1Start;
+            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b2, j2 + (fIdx - 1) * 4);
+            int f2End = IntSerDeUtils.getInt(b2, j2 + fIdx * 4);
+            int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
+            int l2 = f2End - f2Start;
+            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public void close() {
+        this.buffers.clear();
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/IFrameSorter.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/IFrameSorter.java
new file mode 100644
index 0000000..de16aca
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/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.pregelix.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/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/std/sort/IRunGenerator.java
similarity index 61%
copy from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
copy to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/IRunGenerator.java
index fb2d1eb..c193a2d 100644
--- 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/std/sort/IRunGenerator.java
@@ -12,10 +12,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package edu.uci.ics.pregelix.dataflow.std.sort;
 
-package edu.uci.ics.pregelix.dataflow.util;
+import java.util.List;
 
-public enum StorageType {
-    TreeIndex,
-    LSMIndex
-}
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+/**
+ * @author pouria
+ *         Interface for the Run Generator
+ */
+public interface IRunGenerator extends IFrameWriter {
+
+    /**
+     * @return the list of generated (sorted) runs
+     */
+    public List<IFrameReader> getRuns();
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RawBinaryComparator.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RawBinaryComparator.java
new file mode 100644
index 0000000..d6db3c8
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RawBinaryComparator.java
@@ -0,0 +1,35 @@
+/*
+ * 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.std.sort;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+
+public final class RawBinaryComparator implements IBinaryComparator {
+
+    @Override
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+        if (b1 == b2 && s1 == s2) {
+            return 0;
+        }
+        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-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RawNormalizedKeyComputer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RawNormalizedKeyComputer.java
new file mode 100644
index 0000000..f43b499
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RawNormalizedKeyComputer.java
@@ -0,0 +1,52 @@
+/*
+ * 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.std.sort;
+
+public final class RawNormalizedKeyComputer {
+
+    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 ^ Integer.MIN_VALUE;
+    }
+
+    public int normalize2(byte[] bytes, int start, int length) {
+        int nk = 0;
+        for (int i = 4; i < 6; i++) {
+            nk <<= 8;
+            if (i < length) {
+                nk += (bytes[start + i] & 0xff);
+            }
+        }
+        return nk;
+    }
+
+    public int normalize4(byte[] bytes, int start, int length) {
+        int nk = 0;
+        for (int i = 4; i < 8; i++) {
+            nk <<= 8;
+            if (i < length) {
+                nk += (bytes[start + i] & 0xff);
+            }
+        }
+        return nk ^ Integer.MIN_VALUE;
+    }
+}
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ReferencedPriorityQueue.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ReferencedPriorityQueue.java
new file mode 100644
index 0000000..7f2db55
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/ReferencedPriorityQueue.java
@@ -0,0 +1,146 @@
+/*
+ * 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.std.sort;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.pregelix.dataflow.std.sort.RunMergingFrameReader.EntryComparator;
+import edu.uci.ics.pregelix.dataflow.std.util.ReferenceEntry;
+
+public class ReferencedPriorityQueue {
+    private final int frameSize;
+    private final RecordDescriptor recordDescriptor;
+    private final ReferenceEntry entries[];
+    private final int size;
+    private int nItems;
+
+    private final EntryComparator comparator;
+    private final RawNormalizedKeyComputer nmkComputer = new RawNormalizedKeyComputer();
+    private final int[] keyFields;
+
+    public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
+            EntryComparator comparator, int[] keyFields) {
+        this.frameSize = frameSize;
+        this.recordDescriptor = recordDescriptor;
+        if (initSize < 1)
+            throw new IllegalArgumentException();
+        this.comparator = comparator;
+        this.keyFields = keyFields;
+        nItems = initSize;
+        size = (initSize + 1) & 0xfffffffe;
+        entries = new ReferenceEntry[size];
+        for (int i = 0; i < size; i++) {
+            entries[i] = new ReferenceEntry(i, null, -1, keyFields, nmkComputer);
+        }
+        for (int i = initSize; i < size; i++) {
+            entries[i].setExhausted();
+        }
+    }
+
+    /**
+     * Retrieve the top entry without removing it
+     * 
+     * @return the top entry
+     */
+    public ReferenceEntry peek() {
+        return entries[0];
+    }
+
+    /**
+     * compare the new entry with entries within the queue, to find a spot for
+     * this new entry
+     * 
+     * @param entry
+     * @return runid of this entry
+     * @throws IOException
+     */
+    public int popAndReplace(FrameTupleAccessor fta, int tIndex) {
+        ReferenceEntry entry = entries[0];
+        if (entry.getAccessor() == null) {
+            entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
+        }
+        entry.getAccessor().reset(fta.getBuffer());
+        entry.setTupleIndex(tIndex, keyFields, nmkComputer);
+
+        add(entry);
+        return entry.getRunid();
+    }
+
+    /**
+     * Push entry into priority queue
+     * 
+     * @param e
+     *            the new Entry
+     */
+    private void add(ReferenceEntry e) {
+        ReferenceEntry min = entries[0];
+        int slot = (size >> 1) + (min.getRunid() >> 1);
+
+        ReferenceEntry curr = e;
+        while (nItems > 0 && slot > 0) {
+            int c = 0;
+            if (entries[slot].isExhausted()) {
+                // run of entries[slot] is exhausted, i.e. not available, curr
+                // wins
+                c = 1;
+            } else if (entries[slot].getAccessor() != null /*
+                                                            * entries[slot] is
+                                                            * not MIN value
+                                                            */
+                    && !curr.isExhausted() /* curr run is available */) {
+
+                if (curr.getAccessor() != null) {
+                    c = comparator.compare(entries[slot], curr);
+                } else {
+                    // curr is MIN value, wins
+                    c = 1;
+                }
+            }
+
+            if (c <= 0) { // curr lost
+                // entries[slot] swaps up
+                ReferenceEntry tmp = entries[slot];
+                entries[slot] = curr;
+                curr = tmp;// winner to pass up
+            }// else curr wins
+            slot = slot >> 1;
+        }
+        // set new entries[0]
+        entries[0] = curr;
+    }
+
+    /**
+     * Pop is called only when a run is exhausted
+     * 
+     * @return
+     */
+    public ReferenceEntry pop() {
+        ReferenceEntry min = entries[0];
+        min.setExhausted();
+        add(min);
+        nItems--;
+        return min;
+    }
+
+    public boolean areRunsExhausted() {
+        return nItems <= 0;
+    }
+
+    public int size() {
+        return nItems;
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RunMergingFrameReader.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RunMergingFrameReader.java
new file mode 100644
index 0000000..c8dea63
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/sort/RunMergingFrameReader.java
@@ -0,0 +1,178 @@
+/*
+ * 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.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+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.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.pregelix.dataflow.std.util.ReferenceEntry;
+
+public class RunMergingFrameReader implements IFrameReader {
+    private final IHyracksTaskContext ctx;
+    private final IFrameReader[] runCursors;
+    private final List<ByteBuffer> inFrames;
+    private final int[] sortFields;
+    private final RawBinaryComparator[] comparators = new RawBinaryComparator[] { new RawBinaryComparator() };
+    private final RecordDescriptor recordDesc;
+    private final FrameTupleAppender outFrameAppender;
+    private ReferencedPriorityQueue topTuples;
+    private int[] tupleIndexes;
+    private FrameTupleAccessor[] tupleAccessors;
+
+    public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
+            int[] sortFields, RecordDescriptor recordDesc) {
+        this.ctx = ctx;
+        this.runCursors = runCursors;
+        this.inFrames = inFrames;
+        this.sortFields = sortFields;
+        this.recordDesc = recordDesc;
+        outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        tupleAccessors = new FrameTupleAccessor[runCursors.length];
+        EntryComparator comparator = createEntryComparator(comparators);
+        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator,
+                sortFields);
+        tupleIndexes = new int[runCursors.length];
+        for (int i = 0; i < runCursors.length; i++) {
+            tupleIndexes[i] = 0;
+            int runIndex = topTuples.peek().getRunid();
+            runCursors[runIndex].open();
+            if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
+                tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+                tupleAccessors[runIndex].reset(inFrames.get(runIndex));
+                setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+            } else {
+                closeRun(runIndex, runCursors, tupleAccessors);
+                topTuples.pop();
+            }
+        }
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        outFrameAppender.reset(buffer, true);
+        while (!topTuples.areRunsExhausted()) {
+            ReferenceEntry top = topTuples.peek();
+            int runIndex = top.getRunid();
+            FrameTupleAccessor fta = top.getAccessor();
+            int tupleIndex = top.getTupleIndex();
+
+            if (!outFrameAppender.append(fta, tupleIndex)) {
+                return true;
+            }
+
+            ++tupleIndexes[runIndex];
+            setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+        }
+
+        if (outFrameAppender.getTupleCount() > 0) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < runCursors.length; ++i) {
+            closeRun(i, runCursors, tupleAccessors);
+        }
+    }
+
+    private void setNextTopTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
+            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
+        boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+        if (exists) {
+            topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
+        } else {
+            topTuples.pop();
+            closeRun(runIndex, runCursors, tupleAccessors);
+        }
+    }
+
+    private boolean hasNextTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
+            FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
+        if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
+            return false;
+        } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
+            ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
+            if (runCursors[runIndex].nextFrame(buf)) {
+                tupleIndexes[runIndex] = 0;
+                return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+            } else {
+                return false;
+            }
+        } else {
+            return true;
+        }
+    }
+
+    private void closeRun(int index, IFrameReader[] runCursors, IFrameTupleAccessor[] tupleAccessors)
+            throws HyracksDataException {
+        if (runCursors[index] != null) {
+            runCursors[index].close();
+            runCursors[index] = null;
+            tupleAccessors[index] = null;
+        }
+    }
+
+    private EntryComparator createEntryComparator(final RawBinaryComparator[] comparators) {
+        return new EntryComparator();
+    }
+
+    class EntryComparator implements Comparator<ReferenceEntry> {
+
+        @Override
+        public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+            int nmk1 = tp1.getNormalizedKey();
+            int nmk2 = tp2.getNormalizedKey();
+            if (nmk1 != nmk2) {
+                return nmk1 > nmk2 ? 1 : -1;
+            }
+            int nmk3 = tp1.getNormalizedKey4();
+            int nmk4 = tp2.getNormalizedKey4();
+            if (nmk3 != nmk4) {
+                return nmk3 > nmk4 ? 1 : -1;
+            }
+
+            FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
+            FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+            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 c = comparators[f].compare(b1, tPointers1[2 * f + 2], tPointers1[2 * f + 3], b2,
+                        tPointers2[2 * f + 2], tPointers2[2 * f + 3]);
+                if (c != 0) {
+                    return c;
+                }
+            }
+            return 0;
+        }
+
+    }
+}
\ No newline at end of file
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/std/util/CopyUpdateUtil.java
similarity index 98%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/CopyUpdateUtil.java
index 0ff3f04..be2255f 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/std/util/CopyUpdateUtil.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
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/std/util/FunctionProxy.java
similarity index 85%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/FunctionProxy.java
index 5579a77..ee9639a 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/std/util/FunctionProxy.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -39,6 +39,7 @@
     private TupleDeserializer tupleDe;
     private RecordDescriptor inputRd;
     private ClassLoader ctxCL;
+    private boolean initialized = false;
 
     public FunctionProxy(IHyracksTaskContext ctx, IUpdateFunctionFactory functionFactory,
             IRuntimeHookFactory preHookFactory, IRuntimeHookFactory postHookFactory,
@@ -59,11 +60,15 @@
     public void functionOpen() throws HyracksDataException {
         ctxCL = Thread.currentThread().getContextClassLoader();
         Thread.currentThread().setContextClassLoader(ctx.getJobletContext().getClassLoader());
-        inputRd = inputRdFactory.createRecordDescriptor(ctx);
-        tupleDe = new TupleDeserializer(inputRd);
         for (IFrameWriter writer : writers) {
             writer.open();
         }
+
+    }
+
+    private void init() throws HyracksDataException {
+        inputRd = inputRdFactory.createRecordDescriptor(ctx);
+        tupleDe = new TupleDeserializer(inputRd);
         if (preHookFactory != null)
             preHookFactory.createRuntimeHook().configure(ctx);
         function.open(ctx, inputRd, writers);
@@ -82,6 +87,10 @@
      */
     public void functionCall(IFrameTupleAccessor leftAccessor, int leftTupleIndex, ITupleReference right,
             ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) throws HyracksDataException {
+        if (!initialized) {
+            init();
+            initialized = true;
+        }
         Object[] tuple = tupleDe.deserializeRecord(leftAccessor, leftTupleIndex, right);
         function.process(tuple);
         function.update(right, cloneUpdateTb, cursor);
@@ -95,6 +104,10 @@
      */
     public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor)
             throws HyracksDataException {
+        if (!initialized) {
+            init();
+            initialized = true;
+        }
         Object[] tuple = tupleDe.deserializeRecord(updateRef);
         function.process(tuple);
         function.update(updateRef, cloneUpdateTb, cursor);
@@ -110,8 +123,16 @@
      * @throws HyracksDataException
      */
     public void functionCall(ArrayTupleBuilder tb, ITupleReference inPlaceUpdateRef, ArrayTupleBuilder cloneUpdateTb,
-            IIndexCursor cursor) throws HyracksDataException {
-        Object[] tuple = tupleDe.deserializeRecord(tb, inPlaceUpdateRef);
+            IIndexCursor cursor, boolean nullLeft) throws HyracksDataException {
+        if (!initialized) {
+            init();
+            initialized = true;
+        }
+        Object[] tuple = tupleDe.deserializeRecord(tb, inPlaceUpdateRef, nullLeft);
+        if (tuple[1] == null) {
+            /** skip vertice that should not be invoked */
+            return;
+        }
         function.process(tuple);
         function.update(inPlaceUpdateRef, cloneUpdateTb, cursor);
     }
@@ -122,9 +143,11 @@
      * @throws HyracksDataException
      */
     public void functionClose() throws HyracksDataException {
-        if (postHookFactory != null)
-            postHookFactory.createRuntimeHook().configure(ctx);
-        function.close();
+        if (initialized) {
+            if (postHookFactory != null)
+                postHookFactory.createRuntimeHook().configure(ctx);
+            function.close();
+        }
         for (IFrameWriter writer : writers) {
             writer.close();
         }
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ReferenceEntry.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ReferenceEntry.java
new file mode 100644
index 0000000..c22dc34
--- /dev/null
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ReferenceEntry.java
@@ -0,0 +1,93 @@
+/*
+ * 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.std.util;
+
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.pregelix.dataflow.std.sort.RawNormalizedKeyComputer;
+
+public final class ReferenceEntry {
+    private final int runid;
+    private FrameTupleAccessor acccessor;
+    private int tupleIndex;
+    private int[] tPointers;
+    private boolean exhausted = false;
+
+    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+            RawNormalizedKeyComputer nmkComputer) {
+        super();
+        this.runid = runid;
+        this.acccessor = fta;
+        this.tPointers = new int[2 + 2 * keyFields.length];
+        if (fta != null) {
+            initTPointer(fta, tupleIndex, keyFields, nmkComputer);
+        }
+    }
+
+    public int getRunid() {
+        return runid;
+    }
+
+    public FrameTupleAccessor getAccessor() {
+        return acccessor;
+    }
+
+    public void setAccessor(FrameTupleAccessor fta) {
+        this.acccessor = fta;
+    }
+
+    public int[] getTPointers() {
+        return tPointers;
+    }
+
+    public int getTupleIndex() {
+        return tupleIndex;
+    }
+
+    public int getNormalizedKey() {
+        return tPointers[0];
+    }
+
+    public int getNormalizedKey4() {
+        return tPointers[1];
+    }
+
+    public void setTupleIndex(int tupleIndex, int[] keyFields, RawNormalizedKeyComputer nmkComputer) {
+        initTPointer(acccessor, tupleIndex, keyFields, nmkComputer);
+    }
+
+    public void setExhausted() {
+        this.exhausted = true;
+    }
+
+    public boolean isExhausted() {
+        return this.exhausted;
+    }
+
+    private void initTPointer(FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+            RawNormalizedKeyComputer nmkComputer) {
+        this.tupleIndex = tupleIndex;
+        byte[] b1 = fta.getBuffer().array();
+        for (int f = 0; f < keyFields.length; ++f) {
+            int fIdx = keyFields[f];
+            tPointers[2 * f + 2] = fta.getTupleStartOffset(tupleIndex) + fta.getFieldSlotsLength()
+                    + fta.getFieldStartOffset(tupleIndex, fIdx);
+            tPointers[2 * f + 3] = fta.getFieldEndOffset(tupleIndex, fIdx) - fta.getFieldStartOffset(tupleIndex, fIdx);
+            if (f == 0) {
+                tPointers[0] = nmkComputer == null ? 0 : nmkComputer.normalize(b1, tPointers[2], tPointers[3]);
+                tPointers[1] = nmkComputer == null ? 0 : nmkComputer.normalize4(b1, tPointers[2], tPointers[3]);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/ResetableByteArrayInputStream.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ResetableByteArrayInputStream.java
similarity index 96%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/ResetableByteArrayInputStream.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ResetableByteArrayInputStream.java
index 5be9ffc..f6ef7af 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/ResetableByteArrayInputStream.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ResetableByteArrayInputStream.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import java.io.InputStream;
 
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/ResetableByteArrayOutputStream.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ResetableByteArrayOutputStream.java
similarity index 97%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/ResetableByteArrayOutputStream.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ResetableByteArrayOutputStream.java
index a5a20de..ab43a08 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/ResetableByteArrayOutputStream.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/ResetableByteArrayOutputStream.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import java.io.OutputStream;
 import java.util.logging.Level;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/SearchKeyTupleReference.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/SearchKeyTupleReference.java
similarity index 96%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/SearchKeyTupleReference.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/SearchKeyTupleReference.java
index fcefad7..aaa961e 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/SearchKeyTupleReference.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/SearchKeyTupleReference.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
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/std/util/StorageType.java
similarity index 93%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/StorageType.java
index fb2d1eb..af50fbe 100644
--- 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/std/util/StorageType.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 public enum StorageType {
     TreeIndex,
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/TupleDeserializer.java
similarity index 90%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/TupleDeserializer.java
index 2fa1a4b..dd1a64e 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/TupleDeserializer.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/TupleDeserializer.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import java.io.DataInputStream;
 import java.io.IOException;
@@ -110,8 +110,22 @@
         }
     }
 
-    public Object[] deserializeRecord(ArrayTupleBuilder tb, ITupleReference right) throws HyracksDataException {
+    public Object[] deserializeRecord(ArrayTupleBuilder tb, ITupleReference right, boolean nullLeft)
+            throws HyracksDataException {
         try {
+            if (nullLeft) {
+                byte[] rightData = right.getFieldData(1);
+                int rightFieldOffset = right.getFieldStart(1);
+                int rightLen = right.getFieldLength(1);
+                /** skip a halted and no message vertex without deserializing it */
+                if (rightData[rightFieldOffset + rightLen - 1] == 1) {
+                    // halt flag is the last byte of any vertex
+                    record[0] = null;
+                    record[1] = null;
+                    return record;
+                }
+            }
+
             byte[] data = tb.getByteArray();
             int[] offset = tb.getFieldEndOffsets();
             int start = 0;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/UpdateBuffer.java
similarity index 98%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/UpdateBuffer.java
index 4421695..d33334f 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBuffer.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/UpdateBuffer.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/UpdateBufferTupleAccessor.java
similarity index 97%
rename from pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java
rename to pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/UpdateBufferTupleAccessor.java
index f3315d1..4a88c3d 100644
--- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/UpdateBufferTupleAccessor.java
+++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/util/UpdateBufferTupleAccessor.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.dataflow.util;
+package edu.uci.ics.pregelix.dataflow.std.util;
 
 import java.nio.ByteBuffer;
 
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ClearStateOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ClearStateOperatorDescriptor.java
index d86557b..bd9dba7 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ClearStateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ClearStateOperatorDescriptor.java
@@ -32,10 +32,12 @@
 public class ClearStateOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private static final long serialVersionUID = 1L;
     private String jobId;
+    private boolean allStates;
 
-    public ClearStateOperatorDescriptor(JobSpecification spec, String jobId) {
+    public ClearStateOperatorDescriptor(JobSpecification spec, String jobId, boolean allStates) {
         super(spec, 0, 0);
         this.jobId = jobId;
+        this.allStates = allStates;
     }
 
     @Override
@@ -47,7 +49,8 @@
             public void initialize() throws HyracksDataException {
                 RuntimeContext context = (RuntimeContext) ctx.getJobletContext().getApplicationContext()
                         .getApplicationObject();
-                context.clearState(jobId);
+                context.clearState(jobId, allStates);
+                System.gc();
             }
 
             @Override
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java
index 0a9d44d..ca56bb6 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/ConnectorPolicyAssignmentPolicy.java
@@ -25,13 +25,13 @@
 import edu.uci.ics.hyracks.api.dataflow.connectors.SendSideMaterializedBlockingConnectorPolicy;
 import edu.uci.ics.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 
 public class ConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy {
     private static final long serialVersionUID = 1L;
     private IConnectorPolicy senderSideMatPipPolicy = new SendSideMaterializedPipeliningConnectorPolicy();
-    private IConnectorPolicy senderSideMatBlkPolicy = new SendSideMaterializedBlockingConnectorPolicy();
+    //private IConnectorPolicy senderSidePipeliningReceiverSideMatBlkPolicy = new SendSidePipeliningReceiveSideMaterializedBlockingConnectorPolicy();
+    private IConnectorPolicy senderSidePipeliningReceiverSideMatBlkPolicy = new SendSideMaterializedBlockingConnectorPolicy();
     private IConnectorPolicy pipeliningPolicy = new PipeliningConnectorPolicy();
     private JobSpecification spec;
 
@@ -42,14 +42,14 @@
     @Override
     public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
             int[] fanouts) {
-        if (c instanceof MToNPartitioningMergingConnectorDescriptor) {
+        if (c.getClass().getName().contains("MToNPartitioningMergingConnectorDescriptor")) {
             return senderSideMatPipPolicy;
         } else {
             Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> endPoints = spec
                     .getConnectorOperatorMap().get(c.getConnectorId());
             IOperatorDescriptor consumer = endPoints.getRight().getLeft();
             if (consumer instanceof TreeIndexInsertUpdateDeleteOperatorDescriptor) {
-                return senderSideMatBlkPolicy;
+                return senderSidePipeliningReceiverSideMatBlkPolicy;
             } else {
                 return pipeliningPolicy;
             }
diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
index d32cb6b..dc57a09 100644
--- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
+++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java
@@ -39,7 +39,6 @@
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
 import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
-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.IRecordDescriptorFactory;
@@ -114,7 +113,8 @@
                     List<Writable> aggValues = new ArrayList<Writable>();
                     // iterate over hdfs spilled aggregates
                     FileSystem dfs = FileSystem.get(conf);
-                    String spillingDir = BspUtils.getGlobalAggregateSpillingDirName(conf, Vertex.getSuperstep());
+                    String spillingDir = BspUtils.getGlobalAggregateSpillingDirName(conf,
+                            IterationUtils.getSuperstep(BspUtils.getJobId(conf), ctx));
                     FileStatus[] files = dfs.listStatus(new Path(spillingDir));
                     if (files != null) {
                         // goes into this branch only when there are spilled files
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 e16ba48..e444975 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,7 +16,6 @@
 
 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;
@@ -49,6 +48,7 @@
 import edu.uci.ics.pregelix.api.io.VertexReader;
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 
 @SuppressWarnings("rawtypes")
 public class VertexFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -140,14 +140,8 @@
                 ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldSize);
                 DataOutput dos = tb.getDataOutput();
 
-                /**
-                 * set context
-                 */
-                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);
+                IterationUtils.setJobContext(BspUtils.getJobId(conf), ctx, mapperContext);
+                Vertex.taskContext = mapperContext;
 
                 /**
                  * empty vertex value
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
index 9daed12..fd99c30 100644
--- 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
@@ -1,6 +1,5 @@
 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;
@@ -11,7 +10,7 @@
 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;
+import edu.uci.ics.pregelix.api.graph.VertexContext;
 
 public class PJobContext {
     private static final Logger LOGGER = Logger.getLogger(RuntimeContext.class.getName());
@@ -20,6 +19,7 @@
     private final Map<TaskIterationID, IStateObject> appStateMap = new ConcurrentHashMap<TaskIterationID, IStateObject>();
     private Long jobIdToSuperStep;
     private Boolean jobIdToMove;
+    private VertexContext vCtx = new VertexContext();
 
     public void close() throws HyracksDataException {
         for (Entry<Long, List<FileReference>> entry : iterationToFiles.entrySet())
@@ -32,8 +32,11 @@
 
     public void clearState() throws HyracksDataException {
         for (Entry<Long, List<FileReference>> entry : iterationToFiles.entrySet())
-            for (FileReference fileRef : entry.getValue())
-                fileRef.delete();
+            for (FileReference fileRef : entry.getValue()) {
+                if (fileRef != null) {
+                    fileRef.delete();
+                }
+            }
 
         iterationToFiles.clear();
         appStateMap.clear();
@@ -69,7 +72,6 @@
 
             setProperties(numVertices, numEdges, currentIteration, superStep, false, cl);
         }
-        System.gc();
     }
 
     public void recoverVertexProperties(long numVertices, long numEdges, long currentIteration, ClassLoader cl) {
@@ -96,35 +98,31 @@
 
     public void endSuperStep() {
         jobIdToMove = true;
-        LOGGER.info("end iteration " + Vertex.getSuperstep());
+        LOGGER.info("end iteration " + vCtx.getSuperstep());
     }
 
     public Map<Long, List<FileReference>> getIterationToFiles() {
         return iterationToFiles;
     }
 
+    public VertexContext getVertexContext() {
+        return vCtx;
+    }
+
     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);
+                vCtx.setSuperstep(currentIteration);
             } else {
-                //Vertex.setSuperstep(++superStep);
-                superStepMethod.invoke(null, ++superStep);
+                vCtx.setSuperstep(++superStep);
             }
-            //Vertex.setNumVertices(numVertices);
-            numVerticesMethod.invoke(null, numVertices);
-            //Vertex.setNumEdges(numEdges);
-            numEdgesMethod.invoke(null, numEdges);
+            vCtx.setNumVertices(numVertices);
+            vCtx.setNumEdges(numEdges);
+
             jobIdToSuperStep = superStep;
             jobIdToMove = toMove;
-            LOGGER.info("start iteration " + Vertex.getSuperstep());
+            LOGGER.info("start iteration " + vCtx.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 98219d6..a8307d7 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
@@ -21,6 +21,8 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadFactory;
 
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
@@ -44,7 +46,7 @@
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
 import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceRepository;
-import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.graph.VertexContext;
 
 public class RuntimeContext implements IWorkspaceFileFactory {
 
@@ -65,17 +67,17 @@
     };
 
     public RuntimeContext(INCApplicationContext appCtx) {
-        fileMapManager = new TransientFileMapManager();
-        ICacheMemoryAllocator allocator = new HeapBufferAllocator();
-        IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
         int pageSize = 64 * 1024;
         long memSize = Runtime.getRuntime().maxMemory();
         long bufferSize = memSize / 4;
         int numPages = (int) (bufferSize / pageSize);
+
+        fileMapManager = new TransientFileMapManager();
+        ICacheMemoryAllocator allocator = new HeapBufferAllocator();
+        IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator, pageSize, numPages);
         /** let the buffer cache never flush dirty pages */
-        bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs,
-                new PreDelayPageCleanerPolicy(Long.MAX_VALUE), fileMapManager, pageSize, numPages, 1000000,
-                threadFactory);
+        bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), prs, new PreDelayPageCleanerPolicy(
+                Long.MAX_VALUE), fileMapManager, 1000000, threadFactory);
         int numPagesInMemComponents = numPages / 8;
         vbcs = new ArrayList<IVirtualBufferCache>();
         IVirtualBufferCache vBufferCache = new MultitenantVirtualBufferCache(new VirtualBufferCache(
@@ -136,7 +138,7 @@
 
     public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, long currentIteration,
             ClassLoader cl) {
-        PJobContext activeJob = getActiveJob(jobId);
+        PJobContext activeJob = getOrCreateActiveJob(jobId);
         activeJob.setVertexProperties(numVertices, numEdges, currentIteration, cl);
     }
 
@@ -151,14 +153,38 @@
         activeJob.endSuperStep();
     }
 
-    public synchronized void clearState(String jobId) throws HyracksDataException {
+    public synchronized void clearState(String jobId, boolean allStates) throws HyracksDataException {
         PJobContext activeJob = getActiveJob(jobId);
-        activeJob.clearState();
-        activeJobs.remove(jobId);
+        if (activeJob != null) {
+            activeJob.clearState();
+            if (allStates) {
+                activeJobs.remove(jobId);
+            }
+        }
+    }
+
+    public long getSuperstep(String jobId) {
+        PJobContext activeJob = getActiveJob(jobId);
+        return activeJob == null ? 0 : activeJob.getVertexContext().getSuperstep();
+    }
+
+    public void setJobContext(String jobId, TaskAttemptContext tCtx) {
+        PJobContext activeJob = getOrCreateActiveJob(jobId);
+        activeJob.getVertexContext().setContext(tCtx);
+    }
+
+    public VertexContext getVertexContext(String jobId) {
+        PJobContext activeJob = getActiveJob(jobId);
+        return activeJob.getVertexContext();
     }
 
     private PJobContext getActiveJob(String jobId) {
         PJobContext activeJob = activeJobs.get(jobId);
+        return activeJob;
+    }
+
+    private PJobContext getOrCreateActiveJob(String jobId) {
+        PJobContext activeJob = activeJobs.get(jobId);
         if (activeJob == null) {
             activeJob = new PJobContext();
             activeJobs.put(jobId, activeJob);
@@ -170,10 +196,11 @@
     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());
+        long superstep = activeJob.getVertexContext().getSuperstep();
+        List<FileReference> files = activeJob.getIterationToFiles().get(superstep);
         if (files == null) {
             files = new ArrayList<FileReference>();
-            activeJob.getIterationToFiles().put(Vertex.getSuperstep(), files);
+            activeJob.getIterationToFiles().put(superstep, files);
         }
         files.add(fRef);
         return fRef;
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 d834868..4ca67e6 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
@@ -22,14 +22,18 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 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.pregelix.api.graph.VertexContext;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.BspUtils;
 import edu.uci.ics.pregelix.api.util.JobStateUtils;
@@ -39,6 +43,41 @@
 public class IterationUtils {
     public static final String TMP_DIR = BspUtils.TMP_DIR;
 
+    /**
+     * Get the input files' byte size
+     * 
+     * @param job
+     */
+    public static long getInputFileSize(PregelixJob job) {
+        try {
+            Path[] paths = FileInputFormat.getInputPaths(job);
+            FileSystem dfs = FileSystem.get(job.getConfiguration());
+            long size = 0;
+            for (Path path : paths) {
+                FileStatus fstatus = dfs.getFileStatus(path);
+                size += getFileSize(dfs, fstatus.getPath());
+            }
+            return size;
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    private static long getFileSize(FileSystem dfs, Path path) throws IOException {
+        FileStatus fstatus = dfs.getFileStatus(path);
+        if (fstatus.isDir()) {
+            long totalSize = 0;
+            FileStatus[] children = dfs.listStatus(path);
+            for (FileStatus child : children) {
+                Path childPath = child.getPath();
+                totalSize += getFileSize(dfs, childPath);
+            }
+            return totalSize;
+        } else {
+            return fstatus.getLen();
+        }
+    }
+
     public static void setIterationState(IHyracksTaskContext ctx, String pregelixJobId, int partition, int iteration,
             IStateObject state) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
@@ -77,6 +116,24 @@
                 conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration, ctx.getJobletContext().getClassLoader());
     }
 
+    public static long getSuperstep(String pregelixJobId, IHyracksTaskContext ctx) {
+        INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
+        RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
+        return context.getSuperstep(pregelixJobId);
+    }
+
+    public static void setJobContext(String pregelixJobId, IHyracksTaskContext ctx, TaskAttemptContext tCtx) {
+        INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
+        RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
+        context.setJobContext(pregelixJobId, tCtx);
+    }
+
+    public static VertexContext getVertexContext(String pregelixJobId, IHyracksTaskContext ctx) {
+        INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
+        RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
+        return context.getVertexContext(pregelixJobId);
+    }
+
     public static void recoverProperties(String pregelixJobId, IHyracksTaskContext ctx, Configuration conf,
             long currentIteration) {
         INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
@@ -144,12 +201,12 @@
     }
 
     public static Writable readGlobalAggregateValue(Configuration conf, String jobId, String aggClassName)
-    throws HyracksDataException {
+            throws HyracksDataException {
         return BspUtils.readGlobalAggregateValue(conf, jobId, aggClassName);
     }
-    
+
     public static HashMap<String, Writable> readAllGlobalAggregateValues(Configuration conf, String jobId)
-    throws HyracksDataException {
+            throws HyracksDataException {
         return BspUtils.readAllGlobalAggregateValues(conf, jobId);
     }
 
diff --git a/pregelix/pregelix-dist/src/main/resources/scripts/startnc.sh b/pregelix/pregelix-dist/src/main/resources/scripts/startnc.sh
index 8e742ea..f9b6a4e 100644
--- a/pregelix/pregelix-dist/src/main/resources/scripts/startnc.sh
+++ b/pregelix/pregelix-dist/src/main/resources/scripts/startnc.sh
@@ -90,11 +90,13 @@
 #Set JAVA_OPTS
 export JAVA_OPTS=$NCJAVA_OPTS" -Xmx"$MEM_SIZE
 
+#TODO: add the optimized setting for -net-buffer-count
+
 #Launch hyracks nc
 cmd=( "${PREGELIX_HOME}/bin/pregelixnc" )
 cmd+=( -cc-host $CCHOST -cc-port $CC_CLUSTERPORT 
 	   -cluster-net-ip-address $IPADDR -data-ip-address $IPADDR -result-ip-address $IPADDR
-	   -node-id $NODEID -iodevices "${IO_DIRS}" );
+	   -node-id $NODEID -iodevices "${IO_DIRS}" -net-buffer-count 5 );
 
 printf "\n\n\n********************************************\nStarting NC with command %s\n\n" "${cmd[*]}" >> "$NCLOGS_DIR/$NODEID.log"
 ${cmd[@]} >> "$NCLOGS_DIR/$NODEID.log" 2>&1 &
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
index a280c45..4bfa343 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
@@ -17,7 +17,6 @@
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.List;
 
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.Text;
@@ -35,7 +34,7 @@
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.example.client.Client;
 import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
-import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat;
 import edu.uci.ics.pregelix.example.io.VLongWritable;
 
 /**
@@ -53,8 +52,9 @@
         @Override
         public void stepPartial(VLongWritable vertexIndex, VLongWritable msg) throws HyracksDataException {
             long value = msg.get();
-            if (min > value)
+            if (min > value) {
                 min = value;
+            }
         }
 
         @SuppressWarnings({ "rawtypes", "unchecked" })
@@ -66,8 +66,10 @@
 
         @Override
         public void stepFinal(VLongWritable vertexIndex, VLongWritable partialAggregate) throws HyracksDataException {
-            if (min > partialAggregate.get())
-                min = partialAggregate.get();
+            long value = partialAggregate.get();
+            if (min > value) {
+                min = value;
+            }
         }
 
         @Override
@@ -83,50 +85,68 @@
             msgList.add(agg);
             return msgList;
         }
+
+        @Override
+        public void stepPartial2(VLongWritable vertexIndex, VLongWritable partialAggregate) throws HyracksDataException {
+            long value = partialAggregate.get();
+            if (min > value) {
+                min = value;
+            }
+        }
+
+        @Override
+        public VLongWritable finishPartial2() {
+            agg.set(min);
+            return agg;
+        }
     }
 
-    private VLongWritable outputValue = new VLongWritable();
     private VLongWritable tmpVertexValue = new VLongWritable();
-    private long minID;
 
     @Override
     public void compute(Iterator<VLongWritable> msgIterator) {
+        long currentComponent = getVertexValue().get();
+        // First superstep is special, because we can simply look at the neighbors
         if (getSuperstep() == 1) {
-            minID = getVertexId().get();
-            List<Edge<VLongWritable, FloatWritable>> edges = this.getEdges();
-            for (int i = 0; i < edges.size(); i++) {
-                Edge<VLongWritable, FloatWritable> edge = edges.get(i);
+            for (Edge<VLongWritable, FloatWritable> edge : getEdges()) {
                 long neighbor = edge.getDestVertexId().get();
-                if (minID > neighbor) {
-                    minID = neighbor;
+                if (neighbor < currentComponent) {
+                    currentComponent = neighbor;
                 }
             }
-            tmpVertexValue.set(minID);
-            setVertexValue(tmpVertexValue);
-            sendOutMsgs();
-        } else {
-            minID = getVertexId().get();
-            while (msgIterator.hasNext()) {
-                minID = Math.min(minID, msgIterator.next().get());
-            }
-            if (minID < getVertexValue().get()) {
-                tmpVertexValue.set(minID);
+            // Only need to send value if it is not the own id
+            if (currentComponent != getVertexValue().get()) {
+                tmpVertexValue.set(currentComponent);
                 setVertexValue(tmpVertexValue);
-                sendOutMsgs();
+                for (Edge<VLongWritable, FloatWritable> edge : getEdges()) {
+                    VLongWritable neighbor = edge.getDestVertexId();
+                    if (neighbor.get() > currentComponent) {
+                        sendMsg(neighbor, tmpVertexValue);
+                    }
+                }
+            }
+        } else {
+            boolean changed = false;
+            // did we get a smaller id ?
+            while (msgIterator.hasNext()) {
+                VLongWritable message = msgIterator.next();
+                long candidateComponent = message.get();
+                if (candidateComponent < currentComponent) {
+                    currentComponent = candidateComponent;
+                    changed = true;
+                }
+            }
+
+            // propagate new component id to the neighbors
+            if (changed) {
+                tmpVertexValue.set(currentComponent);
+                setVertexValue(tmpVertexValue);
+                sendMsgToAllEdges(tmpVertexValue);
             }
         }
         voteToHalt();
     }
 
-    private void sendOutMsgs() {
-        List<Edge<VLongWritable, FloatWritable>> edges = this.getEdges();
-        outputValue.set(minID);
-        for (int i = 0; i < edges.size(); i++) {
-            Edge<VLongWritable, FloatWritable> edge = edges.get(i);
-            sendMsg(edge.getDestVertexId(), outputValue);
-        }
-    }
-
     @Override
     public String toString() {
         return getVertexId() + " " + getVertexValue();
@@ -135,11 +155,12 @@
     public static void main(String[] args) throws Exception {
         PregelixJob job = new PregelixJob(ConnectedComponentsVertex.class.getSimpleName());
         job.setVertexClass(ConnectedComponentsVertex.class);
-        job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+        job.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class);
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setDynamicVertexValueSize(true);
+        job.setSkipCombinerKey(true);
         Client.run(args, job);
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
index 7fae776..bdf81c7 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphMutationVertex.java
@@ -45,7 +45,7 @@
 
     @Override
     public void compute(Iterator<DoubleWritable> msgIterator) {
-        if (Vertex.getSuperstep() == 1) {
+        if (getSuperstep() == 1) {
             if (newVertex == null) {
                 newVertex = new GraphMutationVertex();
             }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphSampleUndirectedVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphSampleUndirectedVertex.java
new file mode 100644
index 0000000..7e02036
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphSampleUndirectedVertex.java
@@ -0,0 +1,229 @@
+/*
+ * 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.Iterator;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.Edge;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+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.api.io.text.TextVertexOutputFormat.TextVertexWriter;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.api.util.GlobalVertexCountAggregator;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextGraphSampleVertexInputFormat;
+import edu.uci.ics.pregelix.example.io.BooleanWritable;
+import edu.uci.ics.pregelix.example.io.NullWritable;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class GraphSampleUndirectedVertex extends Vertex<VLongWritable, BooleanWritable, BooleanWritable, VLongWritable> {
+
+    public static class GlobalSamplingAggregator
+            extends
+            GlobalAggregator<VLongWritable, BooleanWritable, BooleanWritable, BooleanWritable, LongWritable, LongWritable> {
+
+        private LongWritable state = new LongWritable(0);
+
+        @Override
+        public void init() {
+            state.set(0);
+        }
+
+        @Override
+        public void step(Vertex<VLongWritable, BooleanWritable, BooleanWritable, BooleanWritable> v)
+                throws HyracksDataException {
+            if (v.getVertexValue().get() == true) {
+                state.set(state.get() + 1);
+            }
+        }
+
+        @Override
+        public void step(LongWritable partialResult) {
+            state.set(state.get() + partialResult.get());
+        }
+
+        @Override
+        public LongWritable finishPartial() {
+            return state;
+        }
+
+        @Override
+        public LongWritable finishFinal() {
+            return state;
+        }
+
+    }
+
+    public static final String GLOBAL_RATE = "pregelix.globalrate";
+    private int seedInterval = 0;
+    private int samplingInterval = 2;
+    private float globalRate = 0f;
+
+    private Random random = new Random(System.currentTimeMillis());
+    private BooleanWritable selectedFlag = new BooleanWritable(true);
+    private float fillingRate = 0f;
+
+    @Override
+    public void configure(Configuration conf) {
+        try {
+            globalRate = conf.getFloat(GLOBAL_RATE, 0);
+            seedInterval = (int) (1.0 / (globalRate / 100));
+            if (getSuperstep() > 1) {
+                LongWritable totalSelectedVertex = (LongWritable) IterationUtils.readGlobalAggregateValue(conf,
+                        BspUtils.getJobId(conf), GlobalSamplingAggregator.class.getName());
+                LongWritable totalVertex = (LongWritable) IterationUtils.readGlobalAggregateValue(conf,
+                        BspUtils.getJobId(conf), GlobalVertexCountAggregator.class.getName());
+                fillingRate = (float) totalSelectedVertex.get() / (float) totalVertex.get();
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    public void compute(Iterator<VLongWritable> msgIterator) throws Exception {
+        if (getSuperstep() == 1) {
+            initSeeds();
+        } else {
+            if (fillingRate >= globalRate) {
+                if (msgIterator.hasNext()) {
+                    setVertexValue(selectedFlag);
+                    
+                    //keep the giraph undirected
+                    while (msgIterator.hasNext()) {
+                        //mark the reverse edge
+                        VLongWritable dest = msgIterator.next();
+                        markEdge(dest);
+                    }
+                }
+                voteToHalt();
+            } else {
+                initSeeds();
+                if (msgIterator.hasNext()) {
+                    markAsSelected();
+                }
+                
+                //keep the graph undirected
+                while (msgIterator.hasNext()) {
+                    //mark the reverse edge
+                    VLongWritable dest = msgIterator.next();
+                    markEdge(dest);
+                }
+            }
+        }
+    }
+
+    private void initSeeds() {
+        int randVal = random.nextInt(seedInterval);
+        if (randVal == 0) {
+            markAsSelected();
+        }
+    }
+
+    private void markAsSelected() {
+        setVertexValue(selectedFlag);
+        for (Edge<VLongWritable, BooleanWritable> edge : getEdges()) {
+            int randVal = random.nextInt(samplingInterval);
+            if (randVal == 0) {
+                if (edge.getEdgeValue().get() == false) {
+                    edge.getEdgeValue().set(true);
+                    sendMsg(edge.getDestVertexId(), getVertexId());
+                }
+            }
+        }
+    }
+
+    private void markEdge(VLongWritable destId) {
+        for (Edge<VLongWritable, BooleanWritable> edge : getEdges()) {
+            if (edge.getDestVertexId().equals(destId)) {
+                if (edge.getEdgeValue().get() == false) {
+                    edge.getEdgeValue().set(true);
+                }
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        StringBuffer strBuffer = new StringBuffer();
+        strBuffer.append(getVertexId().toString());
+        strBuffer.append(" ");
+        for (Edge<VLongWritable, BooleanWritable> edge : getEdges()) {
+            if (edge.getEdgeValue().get() == true) {
+                strBuffer.append(edge.getDestVertexId());
+                strBuffer.append(" ");
+            }
+        }
+        return strBuffer.toString().trim();
+    }
+
+    public static void main(String[] args) throws Exception {
+        PregelixJob job = new PregelixJob(GraphSampleUndirectedVertex.class.getSimpleName());
+        job.setVertexClass(GraphSampleUndirectedVertex.class);
+        job.setVertexInputFormatClass(TextGraphSampleVertexInputFormat.class);
+        job.setVertexOutputFormatClass(GraphSampleVertexOutputFormat.class);
+        job.addGlobalAggregatorClass(GraphSampleUndirectedVertex.GlobalSamplingAggregator.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setFixedVertexValueSize(true);
+        job.setSkipCombinerKey(true);
+        Client.run(args, job);
+    }
+
+    /**
+     * write sampled vertices
+     */
+    public static class GraphSampleVertexWriter extends TextVertexWriter<VLongWritable, BooleanWritable, NullWritable> {
+        public GraphSampleVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+            super(lineRecordWriter);
+        }
+
+        @Override
+        public void writeVertex(Vertex<VLongWritable, BooleanWritable, NullWritable, ?> vertex) throws IOException,
+                InterruptedException {
+            if (vertex.getVertexValue().get() == true) {
+                getRecordWriter().write(new Text(vertex.toString()), new Text());
+            }
+        }
+    }
+
+    /**
+     * output format for sampled vertices
+     */
+    public static class GraphSampleVertexOutputFormat extends
+            TextVertexOutputFormat<VLongWritable, BooleanWritable, NullWritable> {
+
+        @Override
+        public VertexWriter<VLongWritable, BooleanWritable, NullWritable> createVertexWriter(TaskAttemptContext context)
+                throws IOException, InterruptedException {
+            RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+            return new GraphSampleVertexWriter(recordWriter);
+        }
+
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphSampleVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphSampleVertex.java
new file mode 100644
index 0000000..bc6a9e4
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/GraphSampleVertex.java
@@ -0,0 +1,252 @@
+/*
+ * 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.Iterator;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.graph.Edge;
+import edu.uci.ics.pregelix.api.graph.GlobalAggregator;
+import edu.uci.ics.pregelix.api.graph.MessageCombiner;
+import edu.uci.ics.pregelix.api.graph.MsgList;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexOutputFormat.TextVertexWriter;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.api.util.GlobalVertexCountAggregator;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
+import edu.uci.ics.pregelix.example.client.Client;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextGraphSampleVertexInputFormat;
+import edu.uci.ics.pregelix.example.io.BooleanWritable;
+import edu.uci.ics.pregelix.example.io.NullWritable;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class GraphSampleVertex extends Vertex<VLongWritable, BooleanWritable, BooleanWritable, BooleanWritable> {
+
+    public static class SimpleSampleCombiner extends MessageCombiner<VLongWritable, BooleanWritable, BooleanWritable> {
+        private BooleanWritable agg = new BooleanWritable();
+        private MsgList<BooleanWritable> msgList;
+
+        @Override
+        public void stepPartial(VLongWritable vertexIndex, BooleanWritable msg) throws HyracksDataException {
+            agg.set(msg.get());
+        }
+
+        @SuppressWarnings({ "rawtypes", "unchecked" })
+        @Override
+        public void init(MsgList msgList) {
+            this.msgList = msgList;
+        }
+
+        @Override
+        public void stepFinal(VLongWritable vertexIndex, BooleanWritable partialAggregate) throws HyracksDataException {
+            agg.set(partialAggregate.get());
+        }
+
+        @Override
+        public BooleanWritable finishPartial() {
+            return agg;
+        }
+
+        @Override
+        public MsgList<BooleanWritable> finishFinal() {
+            msgList.clear();
+            msgList.add(agg);
+            return msgList;
+        }
+
+        @Override
+        public void stepPartial2(VLongWritable vertexIndex, BooleanWritable partialAggregate)
+                throws HyracksDataException {
+            agg.set(partialAggregate.get());
+        }
+
+        @Override
+        public BooleanWritable finishPartial2() {
+            return agg;
+        }
+    }
+
+    public static class GlobalSamplingAggregator
+            extends
+            GlobalAggregator<VLongWritable, BooleanWritable, BooleanWritable, BooleanWritable, LongWritable, LongWritable> {
+
+        private LongWritable state = new LongWritable(0);
+
+        @Override
+        public void init() {
+            state.set(0);
+        }
+
+        @Override
+        public void step(Vertex<VLongWritable, BooleanWritable, BooleanWritable, BooleanWritable> v)
+                throws HyracksDataException {
+            if (v.getVertexValue().get() == true) {
+                state.set(state.get() + 1);
+            }
+        }
+
+        @Override
+        public void step(LongWritable partialResult) {
+            state.set(state.get() + partialResult.get());
+        }
+
+        @Override
+        public LongWritable finishPartial() {
+            return state;
+        }
+
+        @Override
+        public LongWritable finishFinal() {
+            return state;
+        }
+
+    }
+
+    public static final String GLOBAL_RATE = "pregelix.globalrate";
+    private int seedInterval = 0;
+    private int samplingInterval = 2;
+    private float globalRate = 0f;
+
+    private Random random = new Random(System.currentTimeMillis());
+    private BooleanWritable selectedFlag = new BooleanWritable(true);
+    private float fillingRate = 0f;
+
+    @Override
+    public void configure(Configuration conf) {
+        try {
+            globalRate = conf.getFloat(GLOBAL_RATE, 0);
+            seedInterval = (int) (1.0 / (globalRate / 100));
+            if (getSuperstep() > 1) {
+                LongWritable totalSelectedVertex = (LongWritable) IterationUtils.readGlobalAggregateValue(conf,
+                        BspUtils.getJobId(conf), GlobalSamplingAggregator.class.getName());
+                LongWritable totalVertex = (LongWritable) IterationUtils.readGlobalAggregateValue(conf,
+                        BspUtils.getJobId(conf), GlobalVertexCountAggregator.class.getName());
+                fillingRate = (float) totalSelectedVertex.get() / (float) totalVertex.get();
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    public void compute(Iterator<BooleanWritable> msgIterator) throws Exception {
+        if (getSuperstep() == 1) {
+            initSeeds();
+        } else {
+            if (fillingRate >= globalRate) {
+                if (msgIterator.hasNext()) {
+                    setVertexValue(selectedFlag);
+                }
+                voteToHalt();
+            } else {
+                initSeeds();
+                if (msgIterator.hasNext()) {
+                    markAsSelected();
+                }
+            }
+        }
+    }
+
+    private void initSeeds() {
+        int randVal = random.nextInt(seedInterval);
+        if (randVal == 0) {
+            markAsSelected();
+        }
+    }
+
+    private void markAsSelected() {
+        setVertexValue(selectedFlag);
+        for (Edge<VLongWritable, BooleanWritable> edge : getEdges()) {
+            int randVal = random.nextInt(samplingInterval);
+            if (randVal == 0) {
+                if (edge.getEdgeValue().get() == false) {
+                    edge.getEdgeValue().set(true);
+                    sendMsg(edge.getDestVertexId(), selectedFlag);
+                }
+            }
+        }
+    }
+
+    @Override
+    public String toString() {
+        StringBuffer strBuffer = new StringBuffer();
+        strBuffer.append(getVertexId().toString());
+        strBuffer.append(" ");
+        for (Edge<VLongWritable, BooleanWritable> edge : getEdges()) {
+            if (edge.getEdgeValue().get() == true) {
+                strBuffer.append(edge.getDestVertexId());
+                strBuffer.append(" ");
+            }
+        }
+        return strBuffer.toString().trim();
+    }
+
+    public static void main(String[] args) throws Exception {
+        PregelixJob job = new PregelixJob(GraphSampleVertex.class.getSimpleName());
+        job.setVertexClass(GraphSampleVertex.class);
+        job.setVertexInputFormatClass(TextGraphSampleVertexInputFormat.class);
+        job.setVertexOutputFormatClass(GraphSampleVertexOutputFormat.class);
+        job.setMessageCombinerClass(GraphSampleVertex.SimpleSampleCombiner.class);
+        job.addGlobalAggregatorClass(GraphSampleVertex.GlobalSamplingAggregator.class);
+        job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setFixedVertexValueSize(true);
+        job.setSkipCombinerKey(true);
+        Client.run(args, job);
+    }
+
+    /**
+     * write sampled vertices
+     */
+    public static class GraphSampleVertexWriter extends TextVertexWriter<VLongWritable, BooleanWritable, NullWritable> {
+        public GraphSampleVertexWriter(RecordWriter<Text, Text> lineRecordWriter) {
+            super(lineRecordWriter);
+        }
+
+        @Override
+        public void writeVertex(Vertex<VLongWritable, BooleanWritable, NullWritable, ?> vertex) throws IOException,
+                InterruptedException {
+            if (vertex.getVertexValue().get() == true) {
+                getRecordWriter().write(new Text(vertex.toString()), new Text());
+            }
+        }
+    }
+
+    /**
+     * output format for sampled vertices
+     */
+    public static class GraphSampleVertexOutputFormat extends
+            TextVertexOutputFormat<VLongWritable, BooleanWritable, NullWritable> {
+
+        @Override
+        public VertexWriter<VLongWritable, BooleanWritable, NullWritable> createVertexWriter(TaskAttemptContext context)
+                throws IOException, InterruptedException {
+            RecordWriter<Text, Text> recordWriter = textOutputFormat.getRecordWriter(context);
+            return new GraphSampleVertexWriter(recordWriter);
+        }
+
+    }
+}
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 2508a1e..bc4adc6 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
@@ -21,6 +21,7 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
@@ -95,13 +96,32 @@
             msgList.add(agg);
             return msgList;
         }
+
+        @Override
+        public void setPartialCombineState(DoubleWritable combineState) {
+            sum = combineState.get();
+        }
+
+        @Override
+        public void stepPartial2(VLongWritable vertexIndex, DoubleWritable partialAggregate)
+                throws HyracksDataException {
+            sum += partialAggregate.get();
+        }
+
+        @Override
+        public DoubleWritable finishPartial2() {
+            agg.set(sum);
+            return agg;
+        }
+    }
+    
+    @Override
+    public void configure(Configuration conf){
+        maxIteration = conf.getInt(ITERATIONS, 10);
     }
 
     @Override
     public void compute(Iterator<DoubleWritable> msgIterator) {
-        if (maxIteration < 0) {
-            maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
-        }
         if (getSuperstep() == 1) {
             tmpVertexValue.set(1.0 / getNumVertices());
             setVertexValue(tmpVertexValue);
@@ -219,6 +239,7 @@
         job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setFixedVertexValueSize(true);
+        job.setSkipCombinerKey(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 fa16ce5..eecb7de 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
@@ -83,6 +83,17 @@
             msgList.add(agg);
             return msgList;
         }
+
+        @Override
+        public void stepPartial2(VLongWritable vertexIndex, ByteWritable partialAggregate) throws HyracksDataException {
+            int newState = agg.get() | partialAggregate.get();
+            agg.set((byte) newState);
+        }
+
+        @Override
+        public ByteWritable finishPartial2() {
+            return agg;
+        }
     }
 
     private ByteWritable tmpVertexValue = new ByteWritable();
@@ -115,12 +126,14 @@
     private boolean isDest(VLongWritable v) {
         return (v.get() == destId);
     }
+    
+    @Override
+    public void configure(Configuration conf){
+        sourceId = conf.getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+    }
 
     @Override
     public void compute(Iterator<ByteWritable> msgIterator) throws Exception {
-        if (sourceId < 0) {
-            sourceId = getContext().getConfiguration().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
-        }
         if (destId < 0) {
             destId = getContext().getConfiguration().getLong(DEST_ID, DEST_ID_DEFAULT);
         }
@@ -220,6 +233,8 @@
         job.setVertexOutputFormatClass(SimpleReachibilityVertexOutputFormat.class);
         job.setMessageCombinerClass(ReachabilityVertex.SimpleReachibilityCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setSkipCombinerKey(true);
+        job.setFixedVertexValueSize(true);
         Client.run(args, job);
         System.out.println("reachable? " + readReachibilityResult(job.getConfiguration()));
     }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ShortestPathsVertex.java
index 2fea813..80a5c61 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
@@ -17,6 +17,7 @@
 
 import java.util.Iterator;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.FloatWritable;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -47,8 +48,9 @@
         @Override
         public void stepPartial(VLongWritable vertexIndex, DoubleWritable msg) throws HyracksDataException {
             double value = msg.get();
-            if (min > value)
+            if (min > value) {
                 min = value;
+            }
         }
 
         @SuppressWarnings({ "unchecked", "rawtypes" })
@@ -67,8 +69,9 @@
         @Override
         public void stepFinal(VLongWritable vertexIndex, DoubleWritable partialAggregate) throws HyracksDataException {
             double value = partialAggregate.get();
-            if (min > value)
+            if (min > value) {
                 min = value;
+            }
         }
 
         @Override
@@ -78,6 +81,21 @@
             msgList.add(agg);
             return msgList;
         }
+
+        @Override
+        public void stepPartial2(VLongWritable vertexIndex, DoubleWritable partialAggregate)
+                throws HyracksDataException {
+            double value = partialAggregate.get();
+            if (min > value) {
+                min = value;
+            }
+        }
+
+        @Override
+        public DoubleWritable finishPartial2() {
+            agg.set(min);
+            return agg;
+        }
     }
 
     private DoubleWritable outputValue = new DoubleWritable();
@@ -99,10 +117,12 @@
     }
 
     @Override
+    public void configure(Configuration conf) {
+        sourceId = conf.getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+    }
+
+    @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);
@@ -134,7 +154,8 @@
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setMessageCombinerClass(ShortestPathsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
-        job.getConfiguration().setLong(SOURCE_ID, 0);
+        job.setSkipCombinerKey(true);
+        job.setFixedVertexValueSize(true);
         Client.run(args, job);
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
index 9fb0958..3928414 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
@@ -73,6 +73,9 @@
 
         @Option(name = "-dyn-opt", usage = "whether to enable dynamic optimization -- for better performance", required = false)
         public String dynamicOptimization = "false";
+
+        @Option(name = "-cust-prop", usage = "comma separated customized properties, for example: pregelix.xyz=abc,pregelix.efg=hij", required = false)
+        public String customizedProperties = "";
     }
 
     public static void run(String[] args, PregelixJob job) throws Exception {
@@ -133,6 +136,23 @@
         if (options.numIteration > 0)
             job.getConfiguration().setLong(PageRankVertex.ITERATIONS, options.numIteration);
         job.setCheckpointingInterval(options.ckpInterval);
+
+        /**
+         * set customized key value pairs
+         */
+        String customizedProperties = options.customizedProperties;
+        if (customizedProperties.length() > 0) {
+            String[] properties = customizedProperties.split(",");
+            for (String property : properties) {
+                String[] keyValue = property.split("=");
+                if (keyValue.length != 2) {
+                    throw new IllegalStateException(property + " is not a valid key value pair!");
+                }
+                String key = keyValue[0];
+                String value = keyValue[1];
+                job.getConfiguration().set(key, value);
+            }
+        }
     }
 
 }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextConnectedComponentsInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextConnectedComponentsInputFormat.java
index 4062c74..53c9df4 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextConnectedComponentsInputFormat.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextConnectedComponentsInputFormat.java
@@ -17,6 +17,7 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.StringTokenizer;
 
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -38,7 +39,7 @@
     @Override
     public VertexReader<VLongWritable, VLongWritable, FloatWritable, VLongWritable> createVertexReader(
             InputSplit split, TaskAttemptContext context) throws IOException {
-        return new TextReachibilityGraphReader(textInputFormat.createRecordReader(split, context));
+        return new TextConnectedComponentsGraphReader(textInputFormat.createRecordReader(split, context));
     }
 }
 
@@ -46,7 +47,6 @@
 class TextConnectedComponentsGraphReader extends
         TextVertexReader<VLongWritable, VLongWritable, FloatWritable, VLongWritable> {
 
-    private final static String separator = " ";
     private Vertex vertex;
     private VLongWritable vertexId = new VLongWritable();
     private List<VLongWritable> pool = new ArrayList<VLongWritable>();
@@ -73,13 +73,14 @@
 
         vertex.reset();
         Text line = getRecordReader().getCurrentValue();
-        String[] fields = line.toString().split(separator);
+        String lineStr = line.toString();
+        StringTokenizer tokenizer = new StringTokenizer(lineStr);
 
-        if (fields.length > 0) {
+        if (tokenizer.hasMoreTokens()) {
             /**
              * set the src vertex id
              */
-            long src = Long.parseLong(fields[0]);
+            long src = Long.parseLong(tokenizer.nextToken());
             vertexId.set(src);
             vertex.setVertexId(vertexId);
             long dest = -1L;
@@ -87,12 +88,17 @@
             /**
              * set up edges
              */
-            for (int i = 1; i < fields.length; i++) {
-                dest = Long.parseLong(fields[i]);
+            while (tokenizer.hasMoreTokens()) {
+                dest = Long.parseLong(tokenizer.nextToken());
                 VLongWritable destId = allocate();
                 destId.set(dest);
                 vertex.addEdge(destId, null);
             }
+            
+            /**
+             * set the vertex value
+             */
+            vertex.setVertexValue(vertexId);
         }
         // vertex.sortEdges();
         return vertex;
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextGraphSampleVertexInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextGraphSampleVertexInputFormat.java
new file mode 100644
index 0000000..fc676fc
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextGraphSampleVertexInputFormat.java
@@ -0,0 +1,115 @@
+/*
+ * 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.inputformat;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexReader;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat;
+import edu.uci.ics.pregelix.api.io.text.TextVertexInputFormat.TextVertexReader;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.example.io.BooleanWritable;
+import edu.uci.ics.pregelix.example.io.NullWritable;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+public class TextGraphSampleVertexInputFormat extends
+        TextVertexInputFormat<VLongWritable, BooleanWritable, NullWritable, BooleanWritable> {
+
+    @Override
+    public VertexReader<VLongWritable, BooleanWritable, NullWritable, BooleanWritable> createVertexReader(
+            InputSplit split, TaskAttemptContext context) throws IOException {
+        return new TextSampleGraphReader(textInputFormat.createRecordReader(split, context));
+    }
+}
+
+@SuppressWarnings("rawtypes")
+class TextSampleGraphReader extends TextVertexReader<VLongWritable, BooleanWritable, NullWritable, BooleanWritable> {
+
+    private Vertex vertex;
+    private VLongWritable vertexId = new VLongWritable();
+    private List<VLongWritable> pool = new ArrayList<VLongWritable>();
+    private int used = 0;
+    private BooleanWritable value = new BooleanWritable(false);
+
+    public TextSampleGraphReader(RecordReader<LongWritable, Text> lineRecordReader) {
+        super(lineRecordReader);
+    }
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+        return getRecordReader().nextKeyValue();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public Vertex<VLongWritable, BooleanWritable, NullWritable, BooleanWritable> getCurrentVertex() throws IOException,
+            InterruptedException {
+        used = 0;
+        if (vertex == null)
+            vertex = (Vertex) BspUtils.createVertex(getContext().getConfiguration());
+        vertex.getMsgList().clear();
+        vertex.getEdges().clear();
+
+        vertex.reset();
+        Text line = getRecordReader().getCurrentValue();
+        String lineStr = line.toString();
+        StringTokenizer tokenizer = new StringTokenizer(lineStr);
+
+        if (tokenizer.hasMoreTokens()) {
+            /**
+             * set the src vertex id
+             */
+            long src = Long.parseLong(tokenizer.nextToken());
+            vertexId.set(src);
+            vertex.setVertexId(vertexId);
+            long dest = -1L;
+
+            /**
+             * set up edges
+             */
+            while (tokenizer.hasMoreTokens()) {
+                dest = Long.parseLong(tokenizer.nextToken());
+                VLongWritable destId = allocate();
+                destId.set(dest);
+                vertex.addEdge(destId, value);
+            }
+        }
+        vertex.setVertexValue(value);
+        return vertex;
+    }
+
+    private VLongWritable allocate() {
+        if (used >= pool.size()) {
+            VLongWritable value = new VLongWritable();
+            pool.add(value);
+            used++;
+            return value;
+        } else {
+            VLongWritable value = pool.get(used);
+            used++;
+            return value;
+        }
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextPageRankInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextPageRankInputFormat.java
index 67681d3..35207b5 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextPageRankInputFormat.java
@@ -17,6 +17,7 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.StringTokenizer;
 
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -46,7 +47,6 @@
 @SuppressWarnings("rawtypes")
 class TextPageRankGraphReader extends TextVertexReader<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
 
-    private final static String separator = " ";
     private Vertex vertex;
     private VLongWritable vertexId = new VLongWritable();
     private List<VLongWritable> pool = new ArrayList<VLongWritable>();
@@ -73,13 +73,14 @@
 
         vertex.reset();
         Text line = getRecordReader().getCurrentValue();
-        String[] fields = line.toString().split(separator);
+        String lineStr = line.toString();
+        StringTokenizer tokenizer = new StringTokenizer(lineStr);
 
-        if (fields.length > 0) {
+        if (tokenizer.hasMoreTokens()) {
             /**
              * set the src vertex id
              */
-            long src = Long.parseLong(fields[0]);
+            long src = Long.parseLong(tokenizer.nextToken());
             vertexId.set(src);
             vertex.setVertexId(vertexId);
             long dest = -1L;
@@ -87,8 +88,8 @@
             /**
              * set up edges
              */
-            for (int i = 1; i < fields.length; i++) {
-                dest = Long.parseLong(fields[i]);
+            while (tokenizer.hasMoreTokens()) {
+                dest = Long.parseLong(tokenizer.nextToken());
                 VLongWritable destId = allocate();
                 destId.set(dest);
                 vertex.addEdge(destId, null);
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextReachibilityVertexInputFormat.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextReachibilityVertexInputFormat.java
index 5cf6c1c..56de328 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextReachibilityVertexInputFormat.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/inputformat/TextReachibilityVertexInputFormat.java
@@ -17,6 +17,7 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.StringTokenizer;
 
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -43,10 +44,8 @@
 }
 
 @SuppressWarnings("rawtypes")
-class TextReachibilityGraphReader extends
-        TextVertexReader<VLongWritable, VLongWritable, FloatWritable, VLongWritable> {
+class TextReachibilityGraphReader extends TextVertexReader<VLongWritable, VLongWritable, FloatWritable, VLongWritable> {
 
-    private final static String separator = " ";
     private Vertex vertex;
     private VLongWritable vertexId = new VLongWritable();
     private List<VLongWritable> pool = new ArrayList<VLongWritable>();
@@ -73,13 +72,14 @@
 
         vertex.reset();
         Text line = getRecordReader().getCurrentValue();
-        String[] fields = line.toString().split(separator);
+        String lineStr = line.toString();
+        StringTokenizer tokenizer = new StringTokenizer(lineStr);
 
-        if (fields.length > 0) {
+        if (tokenizer.hasMoreTokens()) {
             /**
              * set the src vertex id
              */
-            long src = Long.parseLong(fields[0]);
+            long src = Long.parseLong(tokenizer.nextToken());
             vertexId.set(src);
             vertex.setVertexId(vertexId);
             long dest = -1L;
@@ -87,8 +87,8 @@
             /**
              * set up edges
              */
-            for (int i = 1; i < fields.length; i++) {
-                dest = Long.parseLong(fields[i]);
+            while (tokenizer.hasMoreTokens()) {
+                dest = Long.parseLong(tokenizer.nextToken());
                 VLongWritable destId = allocate();
                 destId.set(dest);
                 vertex.addEdge(destId, null);
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 8987393..caa85bf 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
@@ -17,6 +17,7 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.StringTokenizer;
 
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -47,7 +48,6 @@
 class TextShortestPathsGraphReader extends
         TextVertexReader<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
 
-    private final static String separator = " ";
     private Vertex vertex;
     private FloatWritable initValue = new FloatWritable(1.0f);
     private VLongWritable vertexId = new VLongWritable();
@@ -75,13 +75,14 @@
         vertex.getEdges().clear();
         vertex.reset();
         Text line = getRecordReader().getCurrentValue();
-        String[] fields = line.toString().split(separator);
+        String lineStr = line.toString();
+        StringTokenizer tokenizer = new StringTokenizer(lineStr);
 
-        if (fields.length > 0) {
+        if (tokenizer.hasMoreTokens()) {
             /**
              * set the src vertex id
              */
-            long src = Long.parseLong(fields[0]);
+            long src = Long.parseLong(tokenizer.nextToken());
             vertexId.set(src);
             vertex.setVertexId(vertexId);
             long dest = -1L;
@@ -89,8 +90,8 @@
             /**
              * set up edges
              */
-            for (int i = 1; i < fields.length; i++) {
-                dest = Long.parseLong(fields[i]);
+            while (tokenizer.hasMoreTokens()) {
+                dest = Long.parseLong(tokenizer.nextToken());
                 VLongWritable destId = allocate();
                 destId.set(dest);
                 vertex.addEdge(destId, initValue);
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/DoubleWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/DoubleWritable.java
index ebc7fe4..8c85e3d 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/DoubleWritable.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/DoubleWritable.java
@@ -15,23 +15,69 @@
 
 package edu.uci.ics.pregelix.example.io;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.pregelix.api.io.Pointable;
 import edu.uci.ics.pregelix.api.io.WritableSizable;
+import edu.uci.ics.pregelix.example.utils.SerDeUtils;
 
 /**
  * Writable for Double values.
  */
-public class DoubleWritable extends org.apache.hadoop.io.DoubleWritable implements WritableSizable {
+public class DoubleWritable extends org.apache.hadoop.io.DoubleWritable implements WritableSizable, Pointable {
+
+    private byte[] data = new byte[8];
 
     public DoubleWritable(double value) {
-        super(value);
+        set(value);
     }
 
     public DoubleWritable() {
-        super();
+        set(0.0);
+    }
+
+    public void set(double v) {
+        super.set(v);
+        SerDeUtils.writeLong(Double.doubleToLongBits(v), data, 0);
     }
 
     public int sizeInBytes() {
         return 8;
     }
 
+    @Override
+    public byte[] getByteArray() {
+        return data;
+    }
+
+    @Override
+    public int getStartOffset() {
+        return 0;
+    }
+
+    @Override
+    public int getLength() {
+        return 8;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        SerDeUtils.writeLong(Double.doubleToLongBits(get()), data, 0);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        output.write(data);
+    }
+
+    @Override
+    public int set(byte[] bytes, int offset) {
+        super.set(Double.longBitsToDouble(SerDeUtils.readLong(bytes, offset)));
+        System.arraycopy(bytes, offset, data, 0, 8);
+        return 8;
+    }
+
 }
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
index ffbbff4..d9688bc 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
@@ -16,10 +16,14 @@
 package edu.uci.ics.pregelix.example.io;
 
 import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
 
+import edu.uci.ics.pregelix.api.io.Pointable;
 import edu.uci.ics.pregelix.api.io.WritableSizable;
 import edu.uci.ics.pregelix.example.utils.SerDeUtils;
 
@@ -30,35 +34,121 @@
  * @see org.apache.hadoop.io.WritableUtils#readVLong(DataInput)
  */
 @SuppressWarnings("rawtypes")
-public class VLongWritable extends org.apache.hadoop.io.VLongWritable implements WritableSizable {
+public class VLongWritable extends org.apache.hadoop.io.VLongWritable implements WritableComparable, WritableSizable,
+        Pointable {
+
+    private byte[] data = new byte[10];
+    private int numBytes = -1;
 
     public VLongWritable() {
+        set(0);
     }
 
     public VLongWritable(long value) {
         set(value);
     }
 
+    @Override
+    public void set(long value) {
+        super.set(value);
+        reset();
+    }
+
     public int sizeInBytes() {
-        long i = get();
-        if (i >= -112 && i <= 127) {
-            return 1;
+        return numBytes;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        numBytes = 0;
+        byte firstByte = input.readByte();
+        data[numBytes++] = firstByte;
+        int len = WritableUtils.decodeVIntSize(firstByte);
+        if (len == 1) {
+            super.set(firstByte);
+            return;
+        }
+        long i = 0;
+        input.readFully(data, numBytes, len - 1);
+        numBytes += len - 1;
+        for (int idx = 1; idx < len; idx++) {
+            byte b = data[idx];
+            i = i << 8;
+            i = i | (b & 0xFF);
+        }
+        super.set((WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i));
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        output.write(data, 0, numBytes);
+    }
+
+    @Override
+    public byte[] getByteArray() {
+        return data;
+    }
+
+    @Override
+    public int getStartOffset() {
+        return 0;
+    }
+
+    @Override
+    public int getLength() {
+        return numBytes;
+    }
+
+    @Override
+    public int set(byte[] bytes, int offset) {
+        int position = offset;
+        numBytes = 0;
+        byte firstByte = bytes[position++];
+        data[numBytes++] = firstByte;
+        int len = WritableUtils.decodeVIntSize(firstByte);
+        if (len == 1) {
+            super.set(firstByte);
+            return numBytes;
+        }
+        long i = 0;
+        System.arraycopy(bytes, position, data, numBytes, len - 1);
+        numBytes += len - 1;
+        for (int idx = 1; idx < len; idx++) {
+            byte b = data[idx];
+            i = i << 8;
+            i = i | (b & 0xFF);
+        }
+        super.set((WritableUtils.isNegativeVInt(firstByte) ? (i ^ -1L) : i));
+        return numBytes;
+    }
+
+    private void reset() {
+        numBytes = 0;
+        long value = get();
+        if (value >= -112 && value <= 127) {
+            data[numBytes++] = (byte) value;
+            return;
         }
 
         int len = -112;
-        if (i < 0) {
-            i ^= -1L; // take one's complement'
+        if (value < 0) {
+            value ^= -1L; // take one's complement'
             len = -120;
         }
 
-        long tmp = i;
+        long tmp = value;
         while (tmp != 0) {
             tmp = tmp >> 8;
             len--;
         }
 
+        data[numBytes++] = (byte) len;
         len = (len < -120) ? -(len + 120) : -(len + 112);
-        return len + 1;
+        for (int idx = len; idx != 0; idx--) {
+            int shiftbits = (idx - 1) * 8;
+            long mask = 0xFFL << shiftbits;
+            data[numBytes++] = (byte) ((value & mask) >> shiftbits);
+        }
     }
 
     /** A Comparator optimized for LongWritable. */
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/CommonSource.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/CommonSource.java
new file mode 100644
index 0000000..60602ee
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/CommonSource.java
@@ -0,0 +1,85 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+@SuppressWarnings("deprecation")
+public class CommonSource {
+    public static class MapRecordOnly extends MapReduceBase implements
+            Mapper<LongWritable, Text, LongWritable, NullWritable> {
+
+        public void map(LongWritable id, Text inputValue, OutputCollector<LongWritable, NullWritable> output,
+                Reporter reporter) throws IOException {
+            StringTokenizer tokenizer = new StringTokenizer(inputValue.toString());
+            String key = tokenizer.nextToken();
+            output.collect(new LongWritable(Long.parseLong(key)), NullWritable.get());
+        }
+    }
+
+    public static class ReduceRecordOnly extends MapReduceBase implements
+            Reducer<LongWritable, NullWritable, NullWritable, Text> {
+
+        NullWritable key = NullWritable.get();
+
+        public void reduce(LongWritable inputKey, Iterator<NullWritable> inputValue,
+                OutputCollector<NullWritable, Text> output, Reporter reporter) throws IOException {
+            int counter = 0;
+            while (inputValue.hasNext()) {
+                inputValue.next();
+                counter++;
+            }
+            if (counter >= 5) {
+                output.collect(key, new Text(inputKey.toString()));
+            }
+        }
+    }
+
+    public static void main(String[] args) throws IOException {
+        JobConf job = new JobConf(GraphPreProcessor.class);
+
+        job.setJobName(GraphPreProcessor.class.getSimpleName());
+        job.setMapperClass(MapRecordOnly.class);
+        job.setReducerClass(ReduceRecordOnly.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(NullWritable.class);
+
+        job.setInputFormat(TextInputFormat.class);
+        for (int i = 0; i < args.length - 2; i++) {
+            FileInputFormat.addInputPath(job, new Path(args[i]));
+        }
+        FileOutputFormat.setOutputPath(job, new Path(args[args.length - 2]));
+        job.setNumReduceTasks(Integer.parseInt(args[args.length - 1]));
+        JobClient.runJob(job);
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/DuplicateGraph.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/DuplicateGraph.java
new file mode 100644
index 0000000..5d30143
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/DuplicateGraph.java
@@ -0,0 +1,87 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.TextOutputFormat;
+
+@SuppressWarnings("deprecation")
+public class DuplicateGraph {
+    public static class MapRecordOnly extends MapReduceBase implements Mapper<LongWritable, Text, Text, Text> {
+
+        static long largestId = 172655479;
+        static long largestId2 = 172655479 * 2;
+        static long largestId3 = 172655479 * 3;
+
+        public void map(LongWritable id, Text inputValue, OutputCollector<Text, Text> output, Reporter reporter)
+                throws IOException {
+            StringTokenizer tokenizer = new StringTokenizer(inputValue.toString());
+            String key = tokenizer.nextToken();
+            long keyLong = Long.parseLong(key);
+            String key2 = Long.toString(keyLong + largestId);
+            String key3 = Long.toString(keyLong + largestId2);
+            String key4 = Long.toString(keyLong + largestId3);
+
+            StringBuilder value = new StringBuilder();
+            StringBuilder value2 = new StringBuilder();
+            StringBuilder value3 = new StringBuilder();
+            StringBuilder value4 = new StringBuilder();
+            while (tokenizer.hasMoreTokens()) {
+                String neighbor = tokenizer.nextToken();
+                long neighborLong = Long.parseLong(neighbor);
+                value.append(neighbor + " ");
+                value2.append((neighborLong + largestId) + " ");
+                value3.append((neighborLong + largestId2) + " ");
+                value4.append((neighborLong + largestId3) + " ");
+            }
+            output.collect(new Text(key), new Text(value.toString().trim()));
+            output.collect(new Text(key2), new Text(value2.toString().trim()));
+            output.collect(new Text(key3), new Text(value3.toString().trim()));
+            output.collect(new Text(key4), new Text(value4.toString().trim()));
+        }
+    }
+
+    public static void main(String[] args) throws IOException {
+        JobConf job = new JobConf(DuplicateGraph.class);
+
+        job.setJobName(DuplicateGraph.class.getSimpleName());
+        job.setMapperClass(MapRecordOnly.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(Text.class);
+        job.setInputFormat(TextInputFormat.class);
+        job.setOutputFormat(TextOutputFormat.class);
+
+        job.setInputFormat(TextInputFormat.class);
+        FileInputFormat.setInputPaths(job, args[0]);
+        FileOutputFormat.setOutputPath(job, new Path(args[1]));
+        job.setNumReduceTasks(0);
+        JobClient.runJob(job);
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/FilterCount.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/FilterCount.java
new file mode 100644
index 0000000..06114ac
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/FilterCount.java
@@ -0,0 +1,69 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+@SuppressWarnings("deprecation")
+public class FilterCount {
+    public static class MapRecordOnly extends MapReduceBase implements Mapper<LongWritable, Text, Text, Text> {
+
+        public void map(LongWritable id, Text inputValue, OutputCollector<Text, Text> output, Reporter reporter)
+                throws IOException {
+            StringTokenizer tokenizer = new StringTokenizer(inputValue.toString());
+            String key = tokenizer.nextToken();
+            //skip count
+            tokenizer.nextToken();
+            StringBuilder sb = new StringBuilder();
+            while (tokenizer.hasMoreTokens()) {
+                sb.append(tokenizer.nextToken() + " ");
+            }
+            output.collect(new Text(key), new Text(sb.toString()));
+        }
+    }
+
+    public static void main(String[] args) throws IOException {
+        JobConf job = new JobConf(GraphPreProcessor.class);
+
+        job.setJobName(FilterCount.class.getSimpleName());
+        job.setMapperClass(MapRecordOnly.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(Text.class);
+        job.setInputFormat(TextInputFormat.class);
+
+        job.setInputFormat(TextInputFormat.class);
+        for (int i = 0; i < args.length - 1; i++) {
+            FileInputFormat.addInputPath(job, new Path(args[i]));
+        }
+        FileOutputFormat.setOutputPath(job, new Path(args[args.length - 1]));
+        job.setNumReduceTasks(0);
+        JobClient.runJob(job);
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/FindLargest.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/FindLargest.java
new file mode 100644
index 0000000..2217380
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/FindLargest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+@SuppressWarnings("deprecation")
+public class FindLargest {
+    public static class MapRecordOnly extends MapReduceBase implements
+            Mapper<LongWritable, Text, LongWritable, NullWritable> {
+
+        public void map(LongWritable id, Text inputValue, OutputCollector<LongWritable, NullWritable> output,
+                Reporter reporter) throws IOException {
+            StringTokenizer tokenizer = new StringTokenizer(inputValue.toString());
+            String key = tokenizer.nextToken();
+            output.collect(new LongWritable(Long.parseLong(key)), NullWritable.get());
+        }
+    }
+
+    public static class ReduceRecordOnly extends MapReduceBase implements
+            Reducer<LongWritable, NullWritable, LongWritable, NullWritable> {
+
+        NullWritable value = NullWritable.get();
+        long currentMax = Long.MIN_VALUE;
+        OutputCollector<LongWritable, NullWritable> output;
+
+        public void reduce(LongWritable inputKey, Iterator<NullWritable> inputValue,
+                OutputCollector<LongWritable, NullWritable> output, Reporter reporter) throws IOException {
+            if (this.output == null) {
+                this.output = output;
+            }
+            if (inputKey.get() > currentMax) {
+                currentMax = inputKey.get();
+            }
+        }
+
+        @Override
+        public void close() throws IOException {
+            output.collect(new LongWritable(currentMax), value);
+        }
+    }
+
+    public static void main(String[] args) throws IOException {
+        JobConf job = new JobConf(GraphPreProcessor.class);
+
+        job.setJobName(GraphPreProcessor.class.getSimpleName());
+        job.setMapperClass(MapRecordOnly.class);
+        job.setReducerClass(ReduceRecordOnly.class);
+        job.setCombinerClass(ReduceRecordOnly.class);
+        job.setMapOutputKeyClass(LongWritable.class);
+        job.setMapOutputValueClass(NullWritable.class);
+
+        job.setInputFormat(TextInputFormat.class);
+        for (int i = 0; i < args.length - 2; i++) {
+            FileInputFormat.addInputPath(job, new Path(args[i]));
+        }
+        FileOutputFormat.setOutputPath(job, new Path(args[args.length - 2]));
+        job.setNumReduceTasks(Integer.parseInt(args[args.length - 1]));
+        JobClient.runJob(job);
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/GraphPreProcessor.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/GraphPreProcessor.java
new file mode 100644
index 0000000..02477b1
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/GraphPreProcessor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+@SuppressWarnings("deprecation")
+public class GraphPreProcessor {
+    public static class MapRecordOnly extends MapReduceBase implements Mapper<LongWritable, Text, Text, Text> {
+
+        public void map(LongWritable id, Text inputValue, OutputCollector<Text, Text> output, Reporter reporter)
+                throws IOException {
+            StringTokenizer tokenizer = new StringTokenizer(inputValue.toString());
+            String key = tokenizer.nextToken();
+            //skip the old key
+            tokenizer.nextToken();
+
+            StringBuilder value = new StringBuilder();
+            while (tokenizer.hasMoreTokens()) {
+                value.append(tokenizer.nextToken() + " ");
+            }
+            output.collect(new Text(key), new Text(value.toString().trim()));
+        }
+    }
+
+    public static void main(String[] args) throws IOException {
+        JobConf job = new JobConf(GraphPreProcessor.class);
+
+        job.setJobName(GraphPreProcessor.class.getSimpleName());
+        job.setMapperClass(MapRecordOnly.class);
+        job.setMapOutputKeyClass(Text.class);
+        job.setMapOutputValueClass(Text.class);
+
+        job.setInputFormat(TextInputFormat.class);
+        FileInputFormat.setInputPaths(job, args[0]);
+        FileOutputFormat.setOutputPath(job, new Path(args[1]));
+        job.setNumReduceTasks(0);
+        JobClient.runJob(job);
+    }
+}
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java
index 2800187..897861e 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java
@@ -53,4 +53,35 @@
         return value < -120 || (value >= -112 && value < 0);
     }
 
+    /**
+     * read a long value from an offset
+     * 
+     * @param data
+     * @param offset
+     * @return the long value
+     */
+    public static long readLong(byte[] data, int offset) {
+        return (((long) data[0] << 56) + ((long) (data[1] & 255) << 48)
+                + ((long) (data[2] & 255) << 40) + ((long) (data[3] & 255) << 32)
+                + ((long) (data[4] & 255) << 24) + ((data[5] & 255) << 16) + ((data[6] & 255) << 8) + ((data[7] & 255) << 0));
+    }
+
+    /**
+     * write a long value to a byte region
+     * 
+     * @param v
+     * @param data
+     * @param offset
+     */
+    public static void writeLong(long v, byte[] data, int offset) {
+        data[0] = (byte) (v >>> 56);
+        data[1] = (byte) (v >>> 48);
+        data[2] = (byte) (v >>> 40);
+        data[3] = (byte) (v >>> 32);
+        data[4] = (byte) (v >>> 24);
+        data[5] = (byte) (v >>> 16);
+        data[6] = (byte) (v >>> 8);
+        data[7] = (byte) (v >>> 0);
+    }
+
 }
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureInjectionIterationCompleteHook.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureInjectionIterationCompleteHook.java
new file mode 100644
index 0000000..c59e3ed
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureInjectionIterationCompleteHook.java
@@ -0,0 +1,38 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.pregelix.api.job.IIterationCompleteReporterHook;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+
+/**
+ * @author yingyib
+ */
+public class FailureInjectionIterationCompleteHook implements IIterationCompleteReporterHook {
+
+    @Override
+    public void completeIteration(int superstep, PregelixJob job) throws HyracksDataException {
+        try {
+            if (superstep == 3) {
+                PregelixHyracksIntegrationUtil.shutdownNC1();
+            }
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java
index efc7bcc..7c4ccce 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java
@@ -21,7 +21,6 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.junit.Test;
 
-import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
 import edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner;
@@ -58,26 +57,10 @@
             FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
             job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
             job.setCheckpointHook(ConservativeCheckpointHook.class);
+            job.setIterationCompleteReporterHook(FailureInjectionIterationCompleteHook.class);
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
-            Thread thread = new Thread(new Runnable() {
-
-                @Override
-                public void run() {
-                    try {
-                        synchronized (this) {
-                            while (Vertex.getSuperstep() <= 5) {
-                                this.wait(200);
-                            }
-                            PregelixHyracksIntegrationUtil.shutdownNC1();
-                        }
-                    } catch (Exception e) {
-                        throw new IllegalStateException(e);
-                    }
-                }
-            });
-            thread.start();
             driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
 
             TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
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 421f2f5..886fb58 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
@@ -21,7 +21,6 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.junit.Test;
 
-import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
 import edu.uci.ics.pregelix.core.base.IDriver.Plan;
@@ -56,27 +55,11 @@
             FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
             job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
             job.setCheckpointHook(ConservativeCheckpointHook.class);
-            job.setFixedVertexValueSize(true);
+            job.setIterationCompleteReporterHook(FailureInjectionIterationCompleteHook.class);
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
-            Thread thread = new Thread(new Runnable() {
-
-                @Override
-                public void run() {
-                    try {
-                        synchronized (this) {
-                            while (Vertex.getSuperstep() <= 5) {
-                                this.wait(200);
-                            }
-                            PregelixHyracksIntegrationUtil.shutdownNC1();
-                        }
-                    } catch (Exception e) {
-                        throw new IllegalStateException(e);
-                    }
-                }
-            });
-            thread.start();
+            
             driver.runJob(job, Plan.INNER_JOIN, "127.0.0.1",
                     PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
 
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 b3ad112..c6e85cb 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
@@ -21,7 +21,6 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.junit.Test;
 
-import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
 import edu.uci.ics.pregelix.core.driver.Driver;
@@ -56,26 +55,10 @@
             job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
             job.setCheckpointHook(ConservativeCheckpointHook.class);
             job.setFixedVertexValueSize(true);
+            job.setIterationCompleteReporterHook(FailureInjectionIterationCompleteHook.class);
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
-            Thread thread = new Thread(new Runnable() {
-
-                @Override
-                public void run() {
-                    try {
-                        synchronized (this) {
-                            while (Vertex.getSuperstep() <= 5) {
-                                this.wait(200);
-                            }
-                            PregelixHyracksIntegrationUtil.shutdownNC1();
-                        }
-                    } catch (Exception e) {
-                        throw new IllegalStateException(e);
-                    }
-                }
-            });
-            thread.start();
             driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
 
             TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
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 9a2ef2c..83b896c 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
@@ -21,7 +21,6 @@
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.junit.Test;
 
-import edu.uci.ics.pregelix.api.graph.Vertex;
 import edu.uci.ics.pregelix.api.job.PregelixJob;
 import edu.uci.ics.pregelix.core.driver.Driver;
 import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
@@ -54,26 +53,10 @@
             FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
             job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
             job.setFixedVertexValueSize(true);
+            job.setIterationCompleteReporterHook(FailureInjectionIterationCompleteHook.class);
 
             testCluster.setUp();
             Driver driver = new Driver(PageRankVertex.class);
-            Thread thread = new Thread(new Runnable() {
-
-                @Override
-                public void run() {
-                    try {
-                        synchronized (this) {
-                            while (Vertex.getSuperstep() <= 5) {
-                                this.wait(200);
-                            }
-                            PregelixHyracksIntegrationUtil.shutdownNC1();
-                        }
-                    } catch (Exception e) {
-                        throw new IllegalStateException(e);
-                    }
-                }
-            });
-            thread.start();
             driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
 
             TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/GraphSampleVertexTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/GraphSampleVertexTest.java
new file mode 100644
index 0000000..3afb417
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/GraphSampleVertexTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
+
+import junit.framework.Assert;
+
+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.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.GraphSampleVertex.GraphSampleVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextGraphSampleVertexInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+
+/**
+ * @author yingyib
+ */
+public class GraphSampleVertexTest {
+    private static String INPUTPATH = "data/webmapcomplex";
+    private static String OUTPUTPAH = "actual/result";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+        try {
+            PregelixJob job = new PregelixJob(GraphSampleVertex.class.getName());
+            job.setVertexClass(GraphSampleVertex.class);
+            job.setVertexInputFormatClass(TextGraphSampleVertexInputFormat.class);
+            job.setVertexOutputFormatClass(GraphSampleVertexOutputFormat.class);
+            job.setMessageCombinerClass(GraphSampleVertex.SimpleSampleCombiner.class);
+            job.addGlobalAggregatorClass(GraphSampleVertex.GlobalSamplingAggregator.class);
+            job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+            job.setFixedVertexValueSize(true);
+            job.getConfiguration().set(GraphSampleVertex.GLOBAL_RATE, "0.5f");
+            FileInputFormat.setInputPaths(job, INPUTPATH);
+            FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+
+            testCluster.setUp();
+            Driver driver = new Driver(GraphSampleVertex.class);
+            driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+            int sampledVertexNum = countVertex(OUTPUTPAH);
+            int totalVertexNum = countVertex(INPUTPATH);
+            float ratio = (float) sampledVertexNum / (float) totalVertexNum;
+            Assert.assertEquals(true, ratio >= 0.5f);
+        } finally {
+            PregelixHyracksIntegrationUtil.deinit();
+            testCluster.cleanupHDFS();
+        }
+    }
+
+    private int countVertex(String filePath) throws Exception {
+        File dir = new File(filePath);
+        int count = 0;
+        if (dir.isDirectory()) {
+            File[] files = dir.listFiles();
+            for (File file : files) {
+                if (file.isFile() && !file.getName().contains(".crc")) {
+                    BufferedReader reader = new BufferedReader(new InputStreamReader(new FileInputStream(file)));
+                    while (reader.readLine() != null) {
+                        count++;
+                    }
+                    reader.close();
+                }
+            }
+            return count;
+        } else {
+            return count;
+        }
+    }
+
+}
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
index 65b9845..a5f793f 100644
--- 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
@@ -68,6 +68,10 @@
                 @Override
                 public void run() {
                     try {
+                        synchronized (this) {
+                            this.wait(2000);
+                            this.notifyAll();
+                        }
                         Driver driver = new Driver(PageRankVertex.class);
                         PregelixJob job2 = new PregelixJob(ConnectedComponentsVertex.class.getName());
                         job2.setVertexClass(ConnectedComponentsVertex.class);
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
index cfd1b27..414fab7 100644
--- 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
@@ -65,6 +65,10 @@
                 @Override
                 public void run() {
                     try {
+                        synchronized (this) {
+                            this.wait(5000);
+                            this.notifyAll();
+                        }
                         Driver driver = new Driver(PageRankVertex.class);
                         PregelixJob job2 = new PregelixJob(PageRankVertex.class.getName());
                         job2.setVertexClass(PageRankVertex.class);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
index 5855fd3..9191fad 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/dataload/DataLoadTest.java
@@ -72,6 +72,7 @@
         job.getConfiguration().setClass(PregelixJob.VERTEX_VALUE_CLASS, DoubleWritable.class, Writable.class);
         job.getConfiguration().setClass(PregelixJob.EDGE_VALUE_CLASS, FloatWritable.class, Writable.class);
         job.getConfiguration().setClass(PregelixJob.MESSAGE_VALUE_CLASS, DoubleWritable.class, Writable.class);
+        job.getConfiguration().set(PregelixJob.JOB_ID, "test_job");
     }
 
     public void setUp() throws Exception {
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 c7eff1e..3bedb49 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
@@ -79,7 +79,9 @@
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
         job.setCheckpointHook(ConservativeCheckpointHook.class);
-        job.setEnableDynamicOptimization(true);
+        job.setGroupByAlgorithm(false);
+        job.setGroupByMemoryLimit(3);
+        job.setFrameSize(1024);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
@@ -92,11 +94,11 @@
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
         job.setFixedVertexValueSize(true);
+        job.setSkipCombinerKey(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
         job.setCheckpointHook(ConservativeCheckpointHook.class);
-        job.setEnableDynamicOptimization(true);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
@@ -112,6 +114,7 @@
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
         job.getConfiguration().setLong(ShortestPathsVertex.SOURCE_ID, 0);
         job.setDynamicVertexValueSize(true);
+        job.setSkipCombinerKey(true);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
@@ -122,11 +125,11 @@
         job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setFixedVertexValueSize(true);
+        job.setSkipCombinerKey(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
         job.setCheckpointHook(ConservativeCheckpointHook.class);
-        job.setEnableDynamicOptimization(true);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
@@ -137,11 +140,10 @@
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
-        job.setDynamicVertexValueSize(true);
+        job.setSkipCombinerKey(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
-        job.setEnableDynamicOptimization(true);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
@@ -153,11 +155,10 @@
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
-        job.setDynamicVertexValueSize(true);
+        job.setSkipCombinerKey(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
-        job.setEnableDynamicOptimization(true);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
@@ -168,6 +169,8 @@
         job.setVertexOutputFormatClass(SimpleReachibilityVertexOutputFormat.class);
         job.setMessageCombinerClass(ReachabilityVertex.SimpleReachibilityCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setSkipCombinerKey(true);
+        job.setFixedVertexValueSize(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
@@ -232,7 +235,6 @@
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
-        job.setEnableDynamicOptimization(true);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
index f077053..12195e6 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobrun/RunJobTestCase.java
@@ -94,11 +94,16 @@
     @Test
     public void test() throws Exception {
         setUp();
-        Plan[] plans = new Plan[] { Plan.INNER_JOIN, Plan.OUTER_JOIN, Plan.OUTER_JOIN_SINGLE_SORT, Plan.OUTER_JOIN_SORT };
+        Plan[] plans = new Plan[] { Plan.OUTER_JOIN, Plan.INNER_JOIN };
         for (Plan plan : plans) {
+            job.setMergeConnector(true);
             driver.runJob(job, plan, PregelixHyracksIntegrationUtil.CC_HOST,
                     PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
             compareResults();
+            //job.setMergeConnector(false);
+            //driver.runJob(job, plan, PregelixHyracksIntegrationUtil.CC_HOST,
+            //        PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
+            //compareResults();
         }
         tearDown();
         waitawhile();
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-0
index f1f1d9b..0c89090 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-0
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-0
@@ -1,5 +1,5 @@
-0	0
-4	0
-8	0
-12	0
-16	0
+1	1
+5	1
+9	1
+13	0
+17	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-1
index 0fa02c1..6d2b709 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-1
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-1
@@ -1,5 +1,5 @@
-1	0
-5	0
-9	0
-13	0
-17	0
+2	1
+6	1
+10	1
+14	0
+18	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-2
index 542ccae..f90bfe0 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-2
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-2
@@ -1,5 +1,5 @@
-2	0
-6	0
-10	0
-14	0
-18	0
+3	1
+7	1
+11	0
+15	0
+19	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-3
index 1d5d6d9..503200b 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-3
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsReal/part-3
@@ -1,5 +1,5 @@
-3	0
-7	0
-11	0
-15	0
-19	0
+0	0
+4	1
+8	1
+12	0
+16	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-0
index f1f1d9b..503200b 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-0
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-0
@@ -1,5 +1,5 @@
 0	0
-4	0
-8	0
+4	1
+8	1
 12	0
 16	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-1
index 4e7d87a..4d86486 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-1
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-1
@@ -1,6 +1,6 @@
-1	0
-5	0
-9	0
+1	1
+5	1
+9	1
 13	0
 17	0
 21	21
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-2 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-2
index 542ccae..6d2b709 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-2
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-2
@@ -1,5 +1,5 @@
-2	0
-6	0
-10	0
+2	1
+6	1
+10	1
 14	0
 18	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-3 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-3
index 513f3ff..af3a604 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-3
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex/part-3
@@ -1,5 +1,5 @@
-3	0
-7	0
+3	1
+7	1
 11	0
 15	0
 19	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0
index 2c975de..ca71d2e 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0
@@ -1,9 +1,9 @@
 0	0
-2	0
-4	0
-6	0
-8	0
-10	0
+2	1
+4	1
+6	1
+8	1
+10	1
 12	0
 14	0
 16	0
diff --git a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1 b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1
index 6976bc1..fae4a35 100755
--- a/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1
+++ b/pregelix/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1
@@ -1,8 +1,8 @@
-1	0
-3	0
-5	0
-7	0
-9	0
+1	1
+3	1
+5	1
+7	1
+9	1
 11	0
 13	0
 15	0
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
index 3091c83..1cef17a0 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
@@ -80,7 +80,6 @@
 <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>pregelix.dynamicopt</name><value>true</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>
@@ -122,13 +121,13 @@
 <property><name>ipc.client.idlethreshold</name><value>4000</value></property>
 <property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleConnectedComponentsVertexOutputFormat</value></property>
 <property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>pregelix.skipCombinerKey</name><value>true</value></property>
 <property><name>mapred.map.tasks.speculative.execution</name><value>true</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.TextConnectedComponentsInputFormat</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>true</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/ConnectedComponentsRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
index b6af65c..7b043b8 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
@@ -1,146 +1,145 @@
 <?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
-<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
-<property><name>mapred.task.cache.levels</name><value>2</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>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.ConnectedComponentsVertex$SimpleMinCombiner</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>ConnectedComponents</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>pregelix.incStateLength</name><value>true</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>pregelix.dynamicopt</name><value>true</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.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>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.ConnectedComponentsVertex</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.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>io.serializations</name><value>org.apache.hadoop.io.serializer.WritableSerialization</value></property>
-<property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat</value></property>
-<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
-<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleConnectedComponentsVertexOutputFormat</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.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>ConnectedComponents</value></property>
+<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
+<property><name>local.cache.size</name><value>10737418240</value></property>
+<property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
+<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
+<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.ConnectedComponentsVertex</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>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.ConnectedComponentsVertex$SimpleMinCombiner</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.ConnectedComponentsVertex$SimpleConnectedComponentsVertexOutputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>pregelix.skipCombinerKey</name><value>true</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.TextConnectedComponentsInputFormat</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>mapred.min.split.size</name><value>0</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>ipc.client.tcpnodelay</name><value>false</value></property>
+<property><name>mapred.acls.enabled</name><value>false</value></property>
 </configuration>
\ No newline at end of file
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
index 6fe04fb..857dc48 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml
@@ -23,6 +23,7 @@
 <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>pregelix.framesize</name><value>1024</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>
@@ -45,6 +46,7 @@
 <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>mapred.task.profile.maps</name><value>0-2</value></property>
+<property><name>pregelix.groupmem</name><value>3</value></property>
 <property><name>mapred.userlog.retain.hours</name><value>24</value></property>
 <property><name>pregelix.numVertices</name><value>20</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
@@ -85,7 +87,6 @@
 <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>pregelix.dynamicopt</name><value>true</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>
@@ -115,6 +116,7 @@
 <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>pregelix.groupalg</name><value>false</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>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
index d0f9759..5e1fb16 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml
@@ -81,12 +81,12 @@
 <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>pregelix.skipCombinerKey</name><value>true</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>pregelix.dynamicopt</name><value>true</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>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
index 0173390..c05a4da 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml
@@ -80,7 +80,6 @@
 <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>pregelix.dynamicopt</name><value>true</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>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
index a7a38e0..cd8ee02 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml
@@ -80,7 +80,6 @@
 <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>pregelix.dynamicopt</name><value>true</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>
@@ -122,6 +121,7 @@
 <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>pregelix.skipCombinerKey</name><value>true</value></property>
 <property><name>mapred.map.tasks.speculative.execution</name><value>true</value></property>
 <property><name>mapred.inmem.merge.threshold</name><value>1000</value></property>
 <property><name>hadoop.logfile.size</name><value>10000000</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
index 225429a..8aa6a23 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml
@@ -1,145 +1,147 @@
 <?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>Reachibility</value></property>
-<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
-<property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
-<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
-<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.ReachabilityVertex</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>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>ReachibilityVertex.destId</name><value>10</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.ReachabilityVertex$SimpleReachibilityCombiner</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.ReachabilityVertex$SimpleReachibilityVertexOutputFormat</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>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.TextReachibilityVertexInputFormat</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>ReachibilityVertex.sourceId</name><value>1</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>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.ReachabilityVertex$SimpleReachibilityCombiner</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>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>Reachibility</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>pregelix.skipCombinerKey</name><value>true</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>ReachibilityVertex.sourceId</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.ReachabilityVertex</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>ReachibilityVertex.destId</name><value>10</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.TextReachibilityVertexInputFormat</value></property>
+<property><name>mapred.reduce.copy.backoff</name><value>300</value></property>
+<property><name>pregelix.vertexOutputFormatClass</name><value>edu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityVertexOutputFormat</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/ShortestPathsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
index b757514..41f7588 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.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:/webmap</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>/result</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>ShortestPaths</value></property>
-<property><name>pregelix.nmkComputerClass</name><value>edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer</value></property>
-<property><name>local.cache.size</name><value>10737418240</value></property>
 <property><name>fs.s3n.impl</name><value>org.apache.hadoop.fs.s3native.NativeS3FileSystem</value></property>
-<property><name>mapred.userlog.limit.kb</name><value>0</value></property>
-<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>20</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.ShortestPathsVertex</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>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.ShortestPathsVertex$SimpleMinCombiner</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>SimpleShortestPathsVertex.sourceId</name><value>0</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>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.TextShortestPathsInputFormat</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>true</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>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.ShortestPathsVertex$SimpleMinCombiner</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>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>20</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>ShortestPaths</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>true</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>pregelix.skipCombinerKey</name><value>true</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>SimpleShortestPathsVertex.sourceId</name><value>0</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.ShortestPathsVertex</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>/result</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:/webmap</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.TextShortestPathsInputFormat</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-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml
index 56a52b2..4268444 100644
--- a/pregelix/pregelix-runtime/pom.xml
+++ b/pregelix/pregelix-runtime/pom.xml
@@ -1,18 +1,14 @@
-<!--
- ! 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-runtime</artifactId>
 	<packaging>jar</packaging>
@@ -111,6 +107,8 @@
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-data-std</artifactId>
 			<version>0.2.12-SNAPSHOT</version>
+			<type>jar</type>
+			<scope>compile</scope>
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
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/agg/AccumulatingAggregatorFactory.java
similarity index 88%
rename from pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AccumulatingAggregatorFactory.java
rename to pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/AccumulatingAggregatorFactory.java
index d243c8a..12fb642 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/agg/AccumulatingAggregatorFactory.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.runtime.simpleagg;
+package edu.uci.ics.pregelix.runtime.agg;
 
 import java.nio.ByteBuffer;
 
@@ -29,10 +29,10 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
-import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import edu.uci.ics.pregelix.dataflow.group.IClusteredAggregatorDescriptorFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IAggregateFunction;
 import edu.uci.ics.pregelix.dataflow.std.base.IAggregateFunctionFactory;
+import edu.uci.ics.pregelix.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.std.group.IClusteredAggregatorDescriptorFactory;
 
 public class AccumulatingAggregatorFactory implements IClusteredAggregatorDescriptorFactory {
 
@@ -73,8 +73,8 @@
             }
 
             @Override
-            public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
-                    AggregateState state) throws HyracksDataException {
+            public void init(IFrameTupleAccessor accessor, int tIndex, AggregateState state)
+                    throws HyracksDataException {
                 setGroupKeySize(accessor, tIndex);
                 initAggregateFunctions(state, true);
                 int stateSize = estimateStep(accessor, tIndex, state);
@@ -88,8 +88,8 @@
             }
 
             @Override
-            public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
-                    int stateTupleIndex, AggregateState state) throws HyracksDataException {
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, AggregateState state)
+                    throws HyracksDataException {
                 int stateSize = estimateStep(accessor, tIndex, state);
                 if (stateSize > frameSize) {
                     emitResultTuple(accessor, tIndex, state);
@@ -99,20 +99,31 @@
             }
 
             @Override
-            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
-                    AggregateState state) throws HyracksDataException {
+            public boolean outputFinalResult(IFrameTupleAccessor accessor, int tIndex, AggregateState state,
+                    FrameTupleAppender appender) throws HyracksDataException {
                 Pair<ArrayBackedValueStorage[], IAggregateFunction[]> aggState = (Pair<ArrayBackedValueStorage[], IAggregateFunction[]>) state.state;
                 ArrayBackedValueStorage[] aggOutput = aggState.getLeft();
                 IAggregateFunction[] agg = aggState.getRight();
                 for (int i = 0; i < agg.length; i++) {
                     try {
                         agg[i].finishAll();
-                        tupleBuilder.addField(aggOutput[i].getByteArray(), aggOutput[i].getStartOffset(),
-                                aggOutput[i].getLength());
                     } catch (Exception e) {
                         throw new HyracksDataException(e);
                     }
                 }
+                //write group Keys
+                for (int i = 0; i < groupFields.length; i++) {
+                    if (!appender.appendField(accessor, tIndex, groupFields[i])) {
+                        return false;
+                    }
+                }
+                //write aggregate fields
+                for (int i = 0; i < agg.length; i++) {
+                    if (!appender.appendField(aggOutput[i].getByteArray(), aggOutput[i].getStartOffset(),
+                            aggOutput[i].getLength())) {
+                        return false;
+                    }
+                }
                 return true;
             }
 
@@ -122,8 +133,8 @@
             }
 
             @Override
-            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
-                    AggregateState state) throws HyracksDataException {
+            public boolean outputPartialResult(IFrameTupleAccessor accessor, int tIndex, AggregateState state,
+                    FrameTupleAppender appender) 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/simpleagg/AggregationFunction.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/AggregationFunction.java
similarity index 81%
rename from pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunction.java
rename to pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/AggregationFunction.java
index 5bc30a2..0070c91 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunction.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/AggregationFunction.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.runtime.simpleagg;
+package edu.uci.ics.pregelix.runtime.agg;
 
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -55,6 +55,7 @@
     private Writable combinedResult;
     private MsgList msgList = new MsgList();
     private boolean keyRead = false;
+    private boolean skipKey = false;
 
     public AggregationFunction(IHyracksTaskContext ctx, IConfigurationFactory confFactory, DataOutput tmpOutput,
             IFrameWriter groupByOutputWriter, boolean isFinalStage, boolean partialAggAsInput)
@@ -68,6 +69,7 @@
         combiner = BspUtils.createMessageCombiner(conf);
         key = BspUtils.createVertexIndex(conf);
         value = !partialAggAsInput ? BspUtils.createMessageValue(conf) : BspUtils.createPartialCombineValue(conf);
+        skipKey = BspUtils.getSkipCombinerKey(conf);
     }
 
     @Override
@@ -84,8 +86,12 @@
 
     @Override
     public void step(IFrameTupleReference tuple) throws HyracksDataException {
-        if (!partialAggAsInput) {
-            combiner.stepPartial(key, (WritableSizable) value);
+        if (!isFinalStage) {
+            if (!partialAggAsInput) {
+                combiner.stepPartial(key, (WritableSizable) value);
+            } else {
+                combiner.stepPartial2(key, value);
+            }
         } else {
             combiner.stepFinal(key, value);
         }
@@ -95,12 +101,16 @@
     public void finish() throws HyracksDataException {
         try {
             if (!isFinalStage) {
-                combinedResult = combiner.finishPartial();
+                if (!partialAggAsInput) {
+                    combinedResult = combiner.finishPartial();
+                } else {
+                    combinedResult = combiner.finishPartial2();
+                }
             } else {
                 combinedResult = combiner.finishFinal();
             }
             combinedResult.write(output);
-        } catch (IOException e) {
+        } catch (Exception e) {
             throw new HyracksDataException(e);
         }
     }
@@ -109,7 +119,11 @@
     public void finishAll() throws HyracksDataException {
         try {
             if (!isFinalStage) {
-                combinedResult = combiner.finishPartial();
+                if (!partialAggAsInput) {
+                    combinedResult = combiner.finishPartial();
+                } else {
+                    combinedResult = combiner.finishPartial2();
+                }
             } else {
                 combinedResult = combiner.finishFinalAll();
             }
@@ -134,13 +148,20 @@
         valueInputStream.setByteBuffer(buffer, valueStart);
 
         try {
-            if (!keyRead) {
+            //read key if necessary
+            if (!keyRead && !skipKey) {
                 key.readFields(keyInput);
                 keyRead = true;
             }
+            //read value
             value.readFields(valueInput);
-            if (!partialAggAsInput) {
-                return combiner.estimateAccumulatedStateByteSizePartial(key, (WritableSizable) value);
+
+            if (!isFinalStage) {
+                if (!partialAggAsInput) {
+                    return combiner.estimateAccumulatedStateByteSizePartial(key, (WritableSizable) value);
+                } else {
+                    return combiner.estimateAccumulatedStateByteSizePartial2(key, value);
+                }
             } else {
                 return combiner.estimateAccumulatedStateByteSizeFinal(key, value);
             }
@@ -148,5 +169,4 @@
             throw new HyracksDataException(e);
         }
     }
-
 }
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/AggregationFunctionFactory.java
similarity index 97%
rename from pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunctionFactory.java
rename to pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/AggregationFunctionFactory.java
index 54eccf5..a0deb46 100644
--- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/simpleagg/AggregationFunctionFactory.java
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/AggregationFunctionFactory.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.pregelix.runtime.simpleagg;
+package edu.uci.ics.pregelix.runtime.agg;
 
 import java.io.DataOutput;
 
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregateFunction.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregateFunction.java
new file mode 100644
index 0000000..3906676
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregateFunction.java
@@ -0,0 +1,230 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.pregelix.runtime.agg;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import edu.uci.ics.pregelix.api.graph.MessageCombiner;
+import edu.uci.ics.pregelix.api.graph.MsgList;
+import edu.uci.ics.pregelix.api.io.WritableSizable;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.pregelix.dataflow.std.base.ISerializableAggregateFunction;
+import edu.uci.ics.pregelix.dataflow.std.util.ResetableByteArrayOutputStream;
+
+@SuppressWarnings("rawtypes")
+public class SerializableAggregateFunction implements ISerializableAggregateFunction {
+    private final Configuration conf;
+    private final boolean partialAggAsInput;
+    private MessageCombiner combiner;
+    private ByteBufferInputStream keyInputStream = new ByteBufferInputStream();
+    private ByteBufferInputStream valueInputStream = new ByteBufferInputStream();
+    private ByteBufferInputStream stateInputStream = new ByteBufferInputStream();
+    private DataInput keyInput = new DataInputStream(keyInputStream);
+    private DataInput valueInput = new DataInputStream(valueInputStream);
+    private DataInput stateInput = new DataInputStream(stateInputStream);
+    private ResetableByteArrayOutputStream stateBos = new ResetableByteArrayOutputStream();
+    private DataOutput stateOutput = new DataOutputStream(stateBos);
+    private WritableComparable key;
+    private Writable value;
+    private Writable combinedResult;
+    private Writable finalResult;
+    private MsgList msgList = new MsgList();
+
+    public SerializableAggregateFunction(IHyracksTaskContext ctx, IConfigurationFactory confFactory,
+            boolean partialAggAsInput) throws HyracksDataException {
+        this.conf = confFactory.createConfiguration(ctx);
+        this.partialAggAsInput = partialAggAsInput;
+        msgList.setConf(this.conf);
+
+        combiner = BspUtils.createMessageCombiner(conf);
+        key = BspUtils.createVertexIndex(conf);
+        value = !partialAggAsInput ? BspUtils.createMessageValue(conf) : BspUtils.createPartialCombineValue(conf);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void init(IFrameTupleReference tuple, ArrayTupleBuilder state) throws HyracksDataException {
+        try {
+            /**
+             * bind key and value
+             */
+            bindKeyValue(tuple);
+            key.readFields(keyInput);
+            value.readFields(valueInput);
+
+            combiner.init(msgList);
+
+            /**
+             * call the step function of the aggregator
+             */
+            if (!partialAggAsInput) {
+                combiner.stepPartial(key, (WritableSizable) value);
+            } else {
+                combiner.stepFinal(key, (WritableSizable) value);
+            }
+
+            /**
+             * output state to the array tuple builder
+             */
+            combinedResult = combiner.finishPartial();
+            combinedResult.write(state.getDataOutput());
+            state.addFieldEndOffset();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void step(IFrameTupleReference tuple, IFrameTupleReference state) throws HyracksDataException {
+        try {
+            /**
+             * bind key and value
+             */
+            bindKeyValue(tuple);
+            key.readFields(keyInput);
+            value.readFields(valueInput);
+
+            /**
+             * bind state
+             */
+            bindState(state);
+            combinedResult.readFields(stateInput);
+
+            /**
+             * set the partial state
+             */
+            combiner.setPartialCombineState(combinedResult);
+
+            /**
+             * call the step function of the aggregator
+             */
+            if (!partialAggAsInput) {
+                combiner.stepPartial(key, (WritableSizable) value);
+            } else {
+                combiner.stepFinal(key, (WritableSizable) value);
+            }
+
+            /**
+             * write out partial state
+             */
+            combinedResult = combiner.finishPartial();
+            combinedResult.write(stateOutput);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public void finishPartial(IFrameTupleReference state, ArrayTupleBuilder output) throws HyracksDataException {
+        try {
+            /**
+             * bind state
+             */
+            bindState(state);
+            combinedResult.readFields(stateInput);
+
+            /**
+             * set the partial state
+             */
+            combiner.setPartialCombineState(combinedResult);
+            combinedResult = combiner.finishPartial();
+            combinedResult.write(output.getDataOutput());
+            output.addFieldEndOffset();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public void finishFinal(IFrameTupleReference state, ArrayTupleBuilder output) throws HyracksDataException {
+        try {
+            /**
+             * bind key and value
+             */
+            bindKeyValue(state);
+            key.readFields(keyInput);
+
+            /**
+             * bind state
+             */
+            bindState(state);
+            combinedResult.readFields(stateInput);
+
+            /**
+             * set the partial state
+             */
+            if (!partialAggAsInput) {
+                combiner.setPartialCombineState(combinedResult);
+                combinedResult = combiner.finishPartial();
+                combinedResult.write(output.getDataOutput());
+            } else {
+                combiner.setPartialCombineState(combinedResult);
+                finalResult = combiner.finishFinal();
+                finalResult.write(output.getDataOutput());
+            }
+            output.addFieldEndOffset();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    /**
+     * @param state
+     */
+    private void bindState(IFrameTupleReference state) {
+        FrameTupleReference ftr = (FrameTupleReference) state;
+        IFrameTupleAccessor fta = ftr.getFrameTupleAccessor();
+        ByteBuffer buffer = fta.getBuffer();
+        int tIndex = ftr.getTupleIndex();
+        int combinedStateStart = fta.getFieldSlotsLength() + fta.getTupleStartOffset(tIndex)
+                + fta.getFieldStartOffset(tIndex, 1);
+        stateInputStream.setByteBuffer(buffer, combinedStateStart);
+        stateBos.setByteArray(buffer.array(), combinedStateStart);
+    }
+
+    /**
+     * @param tuple
+     */
+    private void bindKeyValue(IFrameTupleReference tuple) {
+        FrameTupleReference ftr = (FrameTupleReference) tuple;
+        IFrameTupleAccessor fta = ftr.getFrameTupleAccessor();
+        ByteBuffer buffer = fta.getBuffer();
+        int tIndex = ftr.getTupleIndex();
+        int keyStart = fta.getFieldSlotsLength() + fta.getTupleStartOffset(tIndex) + fta.getFieldStartOffset(tIndex, 0);
+        int valueStart = fta.getFieldSlotsLength() + fta.getTupleStartOffset(tIndex)
+                + fta.getFieldStartOffset(tIndex, 1);
+        keyInputStream.setByteBuffer(buffer, keyStart);
+        valueInputStream.setByteBuffer(buffer, valueStart);
+    }
+
+}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregationFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregationFunctionFactory.java
new file mode 100644
index 0000000..c6e41b9
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregationFunctionFactory.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.runtime.agg;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.pregelix.dataflow.std.base.ISerializableAggregateFunction;
+import edu.uci.ics.pregelix.dataflow.std.base.ISerializableAggregateFunctionFactory;
+
+public class SerializableAggregationFunctionFactory implements ISerializableAggregateFunctionFactory {
+    private static final long serialVersionUID = 1L;
+    private final IConfigurationFactory confFactory;
+    private final boolean partialAggAsInput;
+
+    public SerializableAggregationFunctionFactory(IConfigurationFactory confFactory, boolean partialAggAsInput) {
+        this.confFactory = confFactory;
+        this.partialAggAsInput = partialAggAsInput;
+    }
+
+    @Override
+    public ISerializableAggregateFunction createAggregateFunction(IHyracksTaskContext ctx, IFrameWriter writer)
+            throws HyracksException {
+        return new SerializableAggregateFunction(ctx, confFactory, partialAggAsInput);
+    }
+}
diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregatorDescriptorFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregatorDescriptorFactory.java
new file mode 100644
index 0000000..11b7b63
--- /dev/null
+++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/agg/SerializableAggregatorDescriptorFactory.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.runtime.agg;
+
+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.data.accessors.FrameTupleReference;
+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.pregelix.dataflow.std.base.ISerializableAggregateFunction;
+import edu.uci.ics.pregelix.dataflow.std.base.ISerializableAggregateFunctionFactory;
+
+public class SerializableAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
+    private static final long serialVersionUID = 1L;
+    private ISerializableAggregateFunctionFactory aggFuncFactory;
+
+    public SerializableAggregatorDescriptorFactory(ISerializableAggregateFunctionFactory aggFuncFactory) {
+        this.aggFuncFactory = aggFuncFactory;
+    }
+
+    @Override
+    public IAggregatorDescriptor createAggregator(final IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, int[] keyFields, int[] keyFieldsInPartialResults, IFrameWriter writer)
+            throws HyracksDataException {
+        try {
+            final FrameTupleReference tupleRef = new FrameTupleReference();
+            final FrameTupleReference stateRef = new FrameTupleReference();
+            final ISerializableAggregateFunction aggFunc = aggFuncFactory.createAggregateFunction(ctx, writer);
+
+            /**
+             * The serializable version aggregator itself is stateless
+             */
+            return new IAggregatorDescriptor() {
+
+                @Override
+                public AggregateState createAggregateStates() {
+                    return new AggregateState();
+                }
+
+                @Override
+                public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                        AggregateState state) throws HyracksDataException {
+                    tupleRef.reset(accessor, tIndex);
+                    aggFunc.init(tupleRef, tupleBuilder);
+                }
+
+                @Override
+                public void reset() {
+
+                }
+
+                @Override
+                public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
+                        int stateTupleIndex, AggregateState state) throws HyracksDataException {
+                    tupleRef.reset(accessor, tIndex);
+                    stateRef.reset(stateAccessor, stateTupleIndex);
+                    aggFunc.step(tupleRef, stateRef);
+                }
+
+                @Override
+                public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor,
+                        int tIndex, AggregateState state) throws HyracksDataException {
+                    stateRef.reset(accessor, tIndex);
+                    aggFunc.finishPartial(stateRef, tupleBuilder);
+                    return true;
+                }
+
+                @Override
+                public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor,
+                        int tIndex, AggregateState state) throws HyracksDataException {
+                    stateRef.reset(accessor, tIndex);
+                    aggFunc.finishFinal(stateRef, tupleBuilder);
+                    return true;
+                }
+
+                @Override
+                public void close() {
+
+                }
+
+            };
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
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 3e4a811..bd05687 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
@@ -42,7 +42,7 @@
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
-import edu.uci.ics.pregelix.dataflow.util.ResetableByteArrayOutputStream;
+import edu.uci.ics.pregelix.dataflow.std.util.ResetableByteArrayOutputStream;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public class ComputeUpdateFunctionFactory implements IUpdateFunctionFactory {
@@ -107,6 +107,7 @@
             private final List<ArrayTupleBuilder> tbs = new ArrayList<ArrayTupleBuilder>();
             private Configuration conf;
             private boolean dynamicStateLength;
+            private boolean userConfigured;
 
             @Override
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
@@ -115,6 +116,7 @@
                 //LSM index does not have in-place update
                 this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf) || BspUtils.useLSM(conf);
                 this.aggregators = BspUtils.createGlobalAggregators(conf);
+                this.userConfigured = false;
                 for (int i = 0; i < aggregators.size(); i++) {
                     this.aggregators.get(i).init();
                 }
@@ -123,7 +125,7 @@
 
                 this.writerMsg = writers[0];
                 this.bufferMsg = ctx.allocateFrame();
-                this.appenderMsg = new FrameTupleAppender(ctx.getFrameSize());
+                this.appenderMsg = new FrameTupleAppender(ctx.getFrameSize(), 2);
                 this.appenderMsg.reset(bufferMsg, true);
                 this.writers.add(writerMsg);
                 this.appenders.add(appenderMsg);
@@ -155,7 +157,7 @@
                 if (writers.length > 5) {
                     this.writerAlive = writers[5];
                     this.bufferAlive = ctx.allocateFrame();
-                    this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize());
+                    this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize(), 2);
                     this.appenderAlive.reset(bufferAlive, true);
                     this.pushAlive = true;
                     this.writers.add(writerAlive);
@@ -195,6 +197,10 @@
                 }
 
                 try {
+                    if (!userConfigured) {
+                        vertex.configure(conf);
+                        userConfigured = true;
+                    }
                     if (msgContentList.segmentStart()) {
                         vertex.open();
                     }
@@ -239,6 +245,11 @@
 
                 /** write out global aggregate value */
                 writeOutGlobalAggregate();
+
+                /** end of a superstep, for vertices to release resources */
+                if (userConfigured) {
+                    vertex.endSuperstep(conf);
+                }
             }
 
             private void writeOutGlobalAggregate() throws HyracksDataException {
@@ -255,7 +266,7 @@
                     if (!appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(),
                             tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize())) {
                         // aggregate state exceed the page size, write to HDFS
-                        FrameTupleUtils.flushTupleToHDFS(tbGlobalAggregate, conf, Vertex.getSuperstep());
+                        FrameTupleUtils.flushTupleToHDFS(tbGlobalAggregate, conf, vertex.getSuperstep());
                         appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
                     }
                     FrameTupleUtils.flushTuplesFinal(appenderGlobalAggregate, writerGlobalAggregate);
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 9ddcce5..774c180 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
@@ -42,7 +42,7 @@
 import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction;
 import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory;
-import edu.uci.ics.pregelix.dataflow.util.ResetableByteArrayOutputStream;
+import edu.uci.ics.pregelix.dataflow.std.util.ResetableByteArrayOutputStream;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public class StartComputeUpdateFunctionFactory implements IUpdateFunctionFactory {
@@ -110,13 +110,15 @@
             private final List<ArrayTupleBuilder> tbs = new ArrayList<ArrayTupleBuilder>();
             private Configuration conf;
             private boolean dynamicStateLength;
+            private boolean userConfigured;
 
             @Override
             public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers)
                     throws HyracksDataException {
                 this.conf = confFactory.createConfiguration(ctx);
                 //LSM index does not have in-place update
-                this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf) || BspUtils.useLSM(conf);;
+                this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf) || BspUtils.useLSM(conf);
+                this.userConfigured = false;
                 this.aggregators = BspUtils.createGlobalAggregators(conf);
                 for (int i = 0; i < aggregators.size(); i++) {
                     this.aggregators.get(i).init();
@@ -126,7 +128,7 @@
 
                 this.writerMsg = writers[0];
                 this.bufferMsg = ctx.allocateFrame();
-                this.appenderMsg = new FrameTupleAppender(ctx.getFrameSize());
+                this.appenderMsg = new FrameTupleAppender(ctx.getFrameSize(), 2);
                 this.appenderMsg.reset(bufferMsg, true);
                 this.writers.add(writerMsg);
                 this.appenders.add(appenderMsg);
@@ -158,7 +160,7 @@
                 if (writers.length > 5) {
                     this.writerAlive = writers[5];
                     this.bufferAlive = ctx.allocateFrame();
-                    this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize());
+                    this.appenderAlive = new FrameTupleAppender(ctx.getFrameSize(), 2);
                     this.appenderAlive.reset(bufferAlive, true);
                     this.pushAlive = true;
                     this.writers.add(writerAlive);
@@ -192,6 +194,10 @@
                 }
 
                 try {
+                    if (!userConfigured) {
+                        vertex.configure(conf);
+                        userConfigured = true;
+                    }
                     vertex.open();
                     vertex.compute(msgIterator);
                     vertex.close();
@@ -228,6 +234,11 @@
 
                 /** write out global aggregate value */
                 writeOutGlobalAggregate();
+
+                /** end of a superstep, for vertices to release resources */
+                if (userConfigured) {
+                    vertex.endSuperstep(conf);
+                }
             }
 
             private void writeOutGlobalAggregate() throws HyracksDataException {
@@ -244,7 +255,7 @@
                     if (!appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(),
                             tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize())) {
                         // aggregate state exceed the page size, write to HDFS
-                        FrameTupleUtils.flushTupleToHDFS(tbGlobalAggregate, conf, Vertex.getSuperstep());
+                        FrameTupleUtils.flushTupleToHDFS(tbGlobalAggregate, conf, vertex.getSuperstep());
                         appenderGlobalAggregate.reset(bufferGlobalAggregate, true);
                     }
                     FrameTupleUtils.flushTuplesFinal(appenderGlobalAggregate, writerGlobalAggregate);
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 e99fcb3..b7a896d 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
@@ -22,10 +22,14 @@
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 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.graph.Vertex;
 import edu.uci.ics.pregelix.api.util.ArrayListWritable;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 
 public class DatatypeHelper {
     private static final class WritableSerializerDeserializer<T extends Writable> implements ISerializerDeserializer<T> {
@@ -33,11 +37,13 @@
 
         private final Class<T> clazz;
         private transient Configuration conf;
+        private IHyracksTaskContext ctx;
         private T object;
 
-        private WritableSerializerDeserializer(Class<T> clazz, Configuration conf) {
+        private WritableSerializerDeserializer(Class<T> clazz, Configuration conf, IHyracksTaskContext ctx) {
             this.clazz = clazz;
             this.conf = conf;
+            this.ctx = ctx;
         }
 
         @SuppressWarnings({ "unchecked", "rawtypes" })
@@ -49,6 +55,12 @@
             }
             try {
                 T t = clazz.newInstance();
+                if (t instanceof Vertex) {
+                    Vertex vertex = (Vertex) t;
+                    if (vertex.getVertexContext() == null && ctx != null) {
+                        vertex.setVertexContext(IterationUtils.getVertexContext(BspUtils.getJobId(conf), ctx));
+                    }
+                }
                 if (t instanceof ArrayListWritable) {
                     ((ArrayListWritable) t).setConf(conf);
                 }
@@ -87,16 +99,16 @@
 
     @SuppressWarnings({ "rawtypes", "unchecked" })
     public static ISerializerDeserializer<? extends Writable> createSerializerDeserializer(
-            Class<? extends Writable> fClass, Configuration conf) {
-        return new WritableSerializerDeserializer(fClass, conf);
+            Class<? extends Writable> fClass, Configuration conf, IHyracksTaskContext ctx) {
+        return new WritableSerializerDeserializer(fClass, conf, ctx);
     }
 
     public static RecordDescriptor createKeyValueRecordDescriptor(Class<? extends Writable> keyClass,
             Class<? extends Writable> valueClass, Configuration conf) {
         @SuppressWarnings("rawtypes")
         ISerializerDeserializer[] fields = new ISerializerDeserializer[2];
-        fields[0] = createSerializerDeserializer(keyClass, conf);
-        fields[1] = createSerializerDeserializer(valueClass, conf);
+        fields[0] = createSerializerDeserializer(keyClass, conf, null);
+        fields[1] = createSerializerDeserializer(valueClass, conf, null);
         return new RecordDescriptor(fields);
     }
 }
\ 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 3151df2..3489578 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,8 +14,6 @@
  */
 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;
@@ -23,9 +21,11 @@
 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.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;
+import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
 
 public class RuntimeHookFactory implements IRuntimeHookFactory {
 
@@ -48,12 +48,10 @@
                 try {
                     TaskAttemptContext mapperContext = ctxFactory.createContext(conf, new TaskAttemptID());
                     mapperContext.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
-
-                    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);
+                    if(BspUtils.getJobId(conf)==null){
+                        System.out.println("here");
+                    }
+                    IterationUtils.setJobContext(BspUtils.getJobId(conf), ctx, 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 c9b67fb..4c934d3 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
@@ -14,50 +14,45 @@
  */
 package edu.uci.ics.pregelix.runtime.touchpoint;
 
-import java.io.DataInputStream;
-
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 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,
             IConfigurationFactory confFactory) {
-        this.keyIOFactory = keyIOFactory;
-        this.confFactory = confFactory;
     }
 
     public ITuplePartitionComputer createPartitioner() {
         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);
+                    int len = accessor.getFieldLength(tIndex, 0);
+                    return Math.abs(hash(accessor.getBuffer().array(), keyStart, len) % nParts);
+                }
+
+                private int hash(byte[] bytes, int offset, int length) {
+                    int value = 1;
+                    int end = offset + length;
+                    for (int i = offset; i < end; i++)
+                        value = value * 31 + (int) bytes[i];
+                    return value;
                 }
             };
         } 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 c11ac5b..8b89877 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
@@ -31,6 +31,6 @@
     @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     public ISerializerDeserializer getSerializerDeserializer(Configuration conf) {
-        return DatatypeHelper.createSerializerDeserializer(clazz, conf);
+        return DatatypeHelper.createSerializerDeserializer(clazz, conf, null);
     }
 }