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/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);
     }
 }