You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ma...@apache.org on 2013/05/20 19:26:57 UTC

[01/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Updated Branches:
  refs/heads/trunk 9f7a34789 -> 8811165e8


http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java b/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java
deleted file mode 100644
index 0898d63..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.vertex;
-
-import org.apache.giraph.combiner.Combiner;
-import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexValueFactory;
-import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
-import org.apache.giraph.io.formats.JsonBase64VertexInputFormat;
-import org.apache.giraph.io.formats.JsonBase64VertexOutputFormat;
-import org.apache.giraph.job.GiraphConfigurationValidator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.giraph.conf.GiraphConstants.VERTEX_VALUE_FACTORY_CLASS;
-
-
-public class TestVertexTypes {
-
-    /**
-     * Matches the {@link GeneratedVertexInputFormat}
-     */
-    private static class GeneratedVertexMatch extends Vertex<LongWritable,
-        IntWritable, FloatWritable, FloatWritable> {
-        @Override
-        public void compute(Iterable<FloatWritable> messages)
-            throws IOException {
-        }
-    }
-
-    /**
-     * Matches the {@link GeneratedVertexInputFormat}
-     */
-    private static class DerivedVertexMatch extends GeneratedVertexMatch {
-    }
-
-    /**
-     * Mismatches the {@link GeneratedVertexInputFormat}
-     */
-    private static class GeneratedVertexMismatch extends Vertex<LongWritable,
-        FloatWritable, FloatWritable, FloatWritable> {
-        @Override
-        public void compute(Iterable<FloatWritable> messages)
-                throws IOException {
-        }
-    }
-
-    /**
-     * Matches the {@link GeneratedVertexMatch}
-     */
-    private static class GeneratedVertexMatchCombiner extends
-        Combiner<LongWritable, FloatWritable> {
-      @Override
-      public void combine(LongWritable vertexIndex,
-          FloatWritable originalMessage,
-          FloatWritable messageToCombine) {
-      }
-
-      @Override
-      public FloatWritable createInitialMessage() {
-        return null;
-      }
-    }
-
-    /**
-     * Mismatches the {@link GeneratedVertexMatch}
-     */
-    private static class GeneratedVertexMismatchCombiner extends
-        Combiner<LongWritable, DoubleWritable> {
-      @Override
-      public void combine(LongWritable vertexIndex,
-          DoubleWritable originalMessage,
-          DoubleWritable messageToCombine) {
-      }
-
-      @Override
-      public DoubleWritable createInitialMessage() {
-        return null;
-      }
-    }
-
-    /**
-     * Mismatches the {@link GeneratedVertexMatch}
-     */
-    private static class GeneratedVertexMismatchValueFactory implements
-        VertexValueFactory<DoubleWritable> {
-
-      @Override
-      public void initialize(
-          ImmutableClassesGiraphConfiguration<?, DoubleWritable, ?, ?>
-              configuration) {}
-
-      @Override
-      public DoubleWritable createVertexValue() {
-        return new DoubleWritable();
-      }
-    }
-
-    /**
-     * Just populate a conf with testing defaults that won't
-     * upset the GiraphConfigurationValidator.
-     * */
-    private Configuration getDefaultTestConf() {
-      Configuration conf = new Configuration();
-      conf.setInt(GiraphConstants.MAX_WORKERS, 1);
-      conf.setInt(GiraphConstants.MIN_WORKERS, 1);
-      conf.set(GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.getKey(),
-        "org.apache.giraph.io.formats.DUMMY_TEST_VALUE");
-      return conf;
-    }
-
-    @Test
-    public void testMatchingType() throws SecurityException,
-            NoSuchMethodException, NoSuchFieldException {
-        Configuration conf = getDefaultTestConf();
-        GiraphConstants.VERTEX_CLASS.set(conf, GeneratedVertexMatch.class);
-        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
-        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
-            SimpleSuperstepVertexInputFormat.class);
-        GiraphConstants.VERTEX_COMBINER_CLASS.set(conf,
-            GeneratedVertexMatchCombiner.class);
-      @SuppressWarnings("rawtypes")
-      GiraphConfigurationValidator<?, ?, ?, ?> validator =
-        new GiraphConfigurationValidator(conf);
-
-      ImmutableClassesGiraphConfiguration gc = new
-          ImmutableClassesGiraphConfiguration(conf);
-
-
-      validator.validateConfiguration();
-    }
-
-    @Test
-    public void testDerivedMatchingType() throws SecurityException,
-            NoSuchMethodException, NoSuchFieldException {
-        Configuration conf = getDefaultTestConf() ;
-        GiraphConstants.VERTEX_CLASS.set(conf, DerivedVertexMatch.class);
-        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
-        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
-            SimpleSuperstepVertexInputFormat.class);
-        @SuppressWarnings("rawtypes")
-        GiraphConfigurationValidator<?, ?, ?, ?> validator =
-          new GiraphConfigurationValidator(conf);
-        validator.validateConfiguration();
-    }
-
-    @Test
-    public void testDerivedInputFormatType() throws SecurityException,
-            NoSuchMethodException, NoSuchFieldException {
-        Configuration conf = getDefaultTestConf() ;
-        GiraphConstants.VERTEX_CLASS.set(conf, DerivedVertexMatch.class);
-        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
-        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
-            SimpleSuperstepVertexInputFormat.class);
-      @SuppressWarnings("rawtypes")
-      GiraphConfigurationValidator<?, ?, ?, ?> validator =
-        new GiraphConfigurationValidator(conf);
-      validator.validateConfiguration();
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testMismatchingVertex() throws SecurityException,
-      NoSuchMethodException, NoSuchFieldException {
-      Configuration conf = getDefaultTestConf() ;
-      GiraphConstants.VERTEX_CLASS.set(conf, GeneratedVertexMismatch.class);
-      GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
-      GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
-        SimpleSuperstepVertexInputFormat.class);
-      @SuppressWarnings("rawtypes")
-      GiraphConfigurationValidator<?, ?, ?, ?> validator =
-        new GiraphConfigurationValidator(conf);
-      validator.validateConfiguration();
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testMismatchingCombiner() throws SecurityException,
-      NoSuchMethodException, NoSuchFieldException {
-      Configuration conf = getDefaultTestConf() ;
-      GiraphConstants.VERTEX_CLASS.set(conf, GeneratedVertexMatch.class);
-      GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
-      GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
-        SimpleSuperstepVertexInputFormat.class);
-      GiraphConstants.VERTEX_COMBINER_CLASS.set(conf,
-        GeneratedVertexMismatchCombiner.class);
-      @SuppressWarnings("rawtypes")
-      GiraphConfigurationValidator<?, ?, ?, ?> validator =
-        new GiraphConfigurationValidator(conf);
-      validator.validateConfiguration();
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testMismatchingVertexValueFactory() throws SecurityException,
-        NoSuchMethodException, NoSuchFieldException {
-      Configuration conf = getDefaultTestConf() ;
-      GiraphConstants.VERTEX_CLASS.set(conf, GeneratedVertexMatch.class);
-      GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
-      GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
-          SimpleSuperstepVertexInputFormat.class);
-      VERTEX_VALUE_FACTORY_CLASS.set(conf,
-          GeneratedVertexMismatchValueFactory.class);
-      @SuppressWarnings("rawtypes")
-      GiraphConfigurationValidator<?, ?, ?, ?> validator =
-          new GiraphConfigurationValidator(conf);
-      validator.validateConfiguration();
-    }
-
-    @Test
-    public void testJsonBase64FormatType() throws SecurityException,
-            NoSuchMethodException, NoSuchFieldException {
-        Configuration conf = getDefaultTestConf() ;
-        GiraphConstants.VERTEX_CLASS.set(conf, GeneratedVertexMatch.class);
-        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
-        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
-            JsonBase64VertexInputFormat.class);
-        GiraphConstants.VERTEX_OUTPUT_FORMAT_CLASS.set(conf,
-            JsonBase64VertexOutputFormat.class);
-        @SuppressWarnings("rawtypes")
-        GiraphConfigurationValidator<?, ?, ?, ?> validator =
-          new GiraphConfigurationValidator(conf);
-        validator.validateConfiguration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java
index 5091b48..02d2754 100644
--- a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java
+++ b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java
@@ -20,6 +20,7 @@ package org.apache.giraph.io.hbase;
 
 
 import org.apache.giraph.BspCase;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.io.hbase.edgemarker.TableEdgeInputFormat;
 import org.apache.giraph.io.hbase.edgemarker.TableEdgeOutputFormat;
@@ -184,7 +185,7 @@ public class TestHBaseRootMarkerVertextFormat extends BspCase {
       giraphConf.setZooKeeperConfiguration(
           cluster.getMaster().getZooKeeper().getQuorum());
       setupConfiguration(giraphJob);
-      giraphConf.setVertexClass(EdgeNotification.class);
+      giraphConf.setComputationClass(EdgeNotification.class);
       giraphConf.setVertexInputFormatClass(TableEdgeInputFormat.class);
       giraphConf.setVertexOutputFormatClass(TableEdgeOutputFormat.class);
 
@@ -220,16 +221,17 @@ public class TestHBaseRootMarkerVertextFormat extends BspCase {
   The test set only has a 1-1 parent-to-child ratio for this unit test.
    */
   public static class EdgeNotification
-      extends Vertex<Text, Text, Text, Text> {
+      extends BasicComputation<Text, Text, Text, Text> {
     @Override
-    public void compute(Iterable<Text> messages) throws IOException {
+    public void compute(Vertex<Text, Text, Text> vertex,
+        Iterable<Text> messages) throws IOException {
       for (Text message : messages) {
-        getValue().set(message);
+        vertex.getValue().set(message);
       }
       if(getSuperstep() == 0) {
-        sendMessageToAllEdges(getId());
+        sendMessageToAllEdges(vertex, vertex.getId());
       }
-      voteToHalt();
+      vertex.voteToHalt();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
index 5ecb7dd..1af46d1 100644
--- a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
+++ b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
@@ -76,11 +76,11 @@ public class TableEdgeInputFormat extends
      * and it's 'children' qualifier as a single edge.
      */
     @Override
-    public Vertex<Text, Text, Text, ?>
+    public Vertex<Text, Text, Text>
     getCurrentVertex()
         throws IOException, InterruptedException {
       Result row = getRecordReader().getCurrentValue();
-      Vertex<Text, Text, Text, ?> vertex =
+      Vertex<Text, Text, Text> vertex =
           getConf().createVertex();
       Text vertexId = new Text(Bytes.toString(row.getRow()));
       List<Edge<Text, Text>> edges = Lists.newLinkedList();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
index 859f519..a04d386 100644
--- a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
+++ b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
@@ -60,7 +60,7 @@ public class TableEdgeOutputFormat
          Record the vertex value as a the value for a new qualifier 'parent'.
          */
         public void writeVertex(
-                Vertex<Text, Text, Text, ?> vertex)
+                Vertex<Text, Text, Text> vertex)
                 throws IOException, InterruptedException {
               RecordWriter<ImmutableBytesWritable, Writable> writer = getRecordWriter();
               byte[] rowBytes = vertex.getId().getBytes();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatGiraphRunner.java
----------------------------------------------------------------------
diff --git a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatGiraphRunner.java b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatGiraphRunner.java
index 2182a71..6135c7d 100644
--- a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatGiraphRunner.java
+++ b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatGiraphRunner.java
@@ -24,9 +24,9 @@ import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.job.GiraphJob;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.hadoop.conf.Configuration;
@@ -96,9 +96,9 @@ public class HCatGiraphRunner implements Tool {
   private boolean skipOutput = false;
 
   /**
-  * vertex class.
+  * computation class.
   */
-  private Class<? extends Vertex> vertexClass;
+  private Class<? extends Computation> computationClass;
   /**
    * vertex input format internal.
    */
@@ -115,17 +115,17 @@ public class HCatGiraphRunner implements Tool {
   /**
   * Giraph runner class.
    *
-  * @param vertexClass Vertex class
+  * @param computationClass Computation class
   * @param vertexInputFormatClass Vertex input format
   * @param edgeInputFormatClass Edge input format
   * @param vertexOutputFormatClass Output format
   */
   protected HCatGiraphRunner(
-      Class<? extends Vertex> vertexClass,
+      Class<? extends Computation> computationClass,
       Class<? extends VertexInputFormat> vertexInputFormatClass,
       Class<? extends EdgeInputFormat> edgeInputFormatClass,
       Class<? extends VertexOutputFormat> vertexOutputFormatClass) {
-    this.vertexClass = vertexClass;
+    this.computationClass = computationClass;
     this.vertexInputFormatClass = vertexInputFormatClass;
     this.edgeInputFormatClass = edgeInputFormatClass;
     this.vertexOutputFormatClass = vertexOutputFormatClass;
@@ -159,7 +159,7 @@ public class HCatGiraphRunner implements Tool {
 
     // setup GiraphJob
     GiraphJob job = new GiraphJob(getConf(), getClass().getName());
-    job.getConfiguration().setVertexClass(vertexClass);
+    job.getConfiguration().setComputationClass(computationClass);
 
     // setup input from Hive
     if (vertexInputFormatClass != null) {
@@ -240,9 +240,9 @@ public class HCatGiraphRunner implements Tool {
     options.addOption("D", "hiveconf", true,
                 "property=value for Hive/Hadoop configuration");
     options.addOption("w", "workers", true, "Number of workers");
-    if (vertexClass == null) {
-      options.addOption(null, "vertexClass", true,
-          "Giraph Vertex class to use");
+    if (computationClass == null) {
+      options.addOption(null, "computationClass", true,
+          "Giraph Computation class to use");
     }
     if (vertexInputFormatClass == null) {
       options.addOption(null, "vertexInputFormatClass", true,
@@ -282,9 +282,9 @@ public class HCatGiraphRunner implements Tool {
     }
 
     // Giraph classes
-    if (cmdln.hasOption("vertexClass")) {
-      vertexClass = findClass(cmdln.getOptionValue("vertexClass"),
-          Vertex.class);
+    if (cmdln.hasOption("computationClass")) {
+      computationClass = findClass(cmdln.getOptionValue("computationClass"),
+          Computation.class);
     }
     if (cmdln.hasOption("vertexInputFormatClass")) {
       vertexInputFormatClass = findClass(
@@ -307,9 +307,9 @@ public class HCatGiraphRunner implements Tool {
       skipOutput = true;
     }
 
-    if (vertexClass == null) {
+    if (computationClass == null) {
       throw new IllegalArgumentException(
-          "Need the Giraph Vertex class name (-vertexClass) to use");
+          "Need the Giraph Computation class name (-computationClass) to use");
     }
     if (vertexInputFormatClass == null && edgeInputFormatClass == null) {
       throw new IllegalArgumentException(
@@ -454,8 +454,8 @@ public class HCatGiraphRunner implements Tool {
   protected void initGiraphJob(GiraphJob job) {
     LOG.info(getClass().getSimpleName() + " with");
     String prefix = "\t";
-    LOG.info(prefix + "-vertexClass=" +
-         vertexClass.getCanonicalName());
+    LOG.info(prefix + "-computationClass=" +
+         computationClass.getCanonicalName());
     if (vertexInputFormatClass != null) {
       LOG.info(prefix + "-vertexInputFormatClass=" +
           vertexInputFormatClass.getCanonicalName());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
index 08b263f..b5dcdfd 100644
--- a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
+++ b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
@@ -225,10 +225,10 @@ public abstract class HCatalogVertexInputFormat<
     protected abstract Iterable<Edge<I, E>> getEdges(HCatRecord record);
 
     @Override
-    public final Vertex<I, V, E, ?> getCurrentVertex()
+    public final Vertex<I, V, E> getCurrentVertex()
       throws IOException, InterruptedException {
       HCatRecord record = getRecordReader().getCurrentValue();
-      Vertex<I, V, E, ?> vertex = getConf().createVertex();
+      Vertex<I, V, E> vertex = getConf().createVertex();
       vertex.initialize(getVertexId(record), getVertexValue(record),
           getEdges(record));
       ++recordCount;
@@ -287,7 +287,7 @@ public abstract class HCatalogVertexInputFormat<
     /**
      * vertex.
      */
-    private Vertex<I, V, E, ?> vertex = null;
+    private Vertex<I, V, E> vertex = null;
     /**
      * Timed logger to print every 30 seconds
      */
@@ -328,7 +328,7 @@ public abstract class HCatalogVertexInputFormat<
     protected abstract E getEdgeValue(HCatRecord record);
 
     @Override
-    public final Vertex<I, V, E, ?>
+    public final Vertex<I, V, E>
     getCurrentVertex() throws IOException, InterruptedException {
       return vertex;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
index 37b3b74..1d234d1 100644
--- a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
+++ b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
@@ -176,21 +176,21 @@ public abstract class HCatalogVertexOutputFormat<
     * @param vertex to populate record
     */
     protected abstract void fillRecord(HCatRecord record,
-                                    Vertex<I, V, E, ?> vertex);
+        Vertex<I, V, E> vertex);
 
     /**
     * create record
     * @param vertex to populate record
     * @return HCatRecord newly created
     */
-    protected HCatRecord createRecord(Vertex<I, V, E, ?> vertex) {
+    protected HCatRecord createRecord(Vertex<I, V, E> vertex) {
       HCatRecord record = new DefaultHCatRecord(getNumColumns());
       fillRecord(record, vertex);
       return record;
     }
 
     @Override
-    public final void writeVertex(Vertex<I, V, E, ?> vertex) throws IOException,
+    public final void writeVertex(Vertex<I, V, E> vertex) throws IOException,
         InterruptedException {
       getRecordWriter().write(null, createRecord(vertex));
     }
@@ -215,10 +215,10 @@ public abstract class HCatalogVertexOutputFormat<
     * @return Iterable of records
     */
     protected abstract Iterable<HCatRecord> createRecords(
-        Vertex<I, V, E, ?> vertex);
+        Vertex<I, V, E> vertex);
 
     @Override
-    public final void writeVertex(Vertex<I, V, E, ?> vertex) throws IOException,
+    public final void writeVertex(Vertex<I, V, E> vertex) throws IOException,
         InterruptedException {
       Iterable<HCatRecord> records = createRecords(vertex);
       for (HCatRecord record : records) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/HiveGiraphRunner.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/HiveGiraphRunner.java b/giraph-hive/src/main/java/org/apache/giraph/hive/HiveGiraphRunner.java
index da9ee2f..340b461 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/HiveGiraphRunner.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/HiveGiraphRunner.java
@@ -27,7 +27,7 @@ import org.apache.commons.cli.ParseException;
 import org.apache.giraph.conf.GiraphClasses;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.hive.input.edge.HiveEdgeInputFormat;
 import org.apache.giraph.hive.input.edge.HiveToEdge;
 import org.apache.giraph.hive.input.vertex.HiveToVertex;
@@ -78,7 +78,7 @@ public class HiveGiraphRunner implements Tool {
   protected boolean isVerbose;
 
   /** vertex class. */
-  private Class<? extends Vertex> vertexClass;
+  private Class<? extends Computation> computationClass;
 
   /** Descriptions of vertex input formats */
   private List<VertexInputFormatDescription> vertexInputDescriptions =
@@ -101,12 +101,13 @@ public class HiveGiraphRunner implements Tool {
     conf = new HiveConf(getClass());
   }
 
-  public Class<? extends Vertex> getVertexClass() {
-    return vertexClass;
+  public Class<? extends Computation> getComputationClass() {
+    return computationClass;
   }
 
-  public void setVertexClass(Class<? extends Vertex> vertexClass) {
-    this.vertexClass = vertexClass;
+  public void setComputationClass(
+      Class<? extends Computation> computationClass) {
+    this.computationClass = computationClass;
   }
 
   public List<VertexInputFormatDescription> getVertexInputDescriptions() {
@@ -265,7 +266,7 @@ public class HiveGiraphRunner implements Tool {
     // setup GiraphJob
     GiraphJob job = new GiraphJob(getConf(), getClass().getName());
     GiraphConfiguration giraphConf = job.getConfiguration();
-    giraphConf.setVertexClass(vertexClass);
+    giraphConf.setComputationClass(computationClass);
 
     giraphConf.setWorkerConfiguration(workers, workers, 100.0f);
     initGiraphJob(job);
@@ -363,14 +364,14 @@ public class HiveGiraphRunner implements Tool {
     }
 
     // Giraph classes
-    String vertexClassStr = cmdln.getOptionValue("vertexClass");
-    if (vertexClassStr != null) {
-      vertexClass = findClass(vertexClassStr, Vertex.class);
+    String computationClassStr = cmdln.getOptionValue("computationClass");
+    if (computationClassStr != null) {
+      computationClass = findClass(computationClassStr, Computation.class);
     }
-    if (vertexClass == null) {
+    if (computationClass == null) {
       throw new IllegalArgumentException(
-          "Need the Giraph " + Vertex.class.getSimpleName() +
-              " class name (-vertexClass) to use");
+          "Need the Giraph " + Computation.class.getSimpleName() +
+              " class name (-computationClass) to use");
     }
 
     String[] vertexInputs = cmdln.getOptionValues("vertexInput");
@@ -502,9 +503,9 @@ public class HiveGiraphRunner implements Tool {
                 "property=value for Hive/Hadoop configuration");
     options.addOption("w", "workers", true, "Number of workers");
 
-    if (vertexClass == null) {
-      options.addOption(null, "vertexClass", true,
-          "Giraph Vertex class to use");
+    if (computationClass == null) {
+      options.addOption(null, "computationClass", true,
+          "Giraph Computation class to use");
     }
 
     options.addOption("db", "dbName", true, "Hive database name");
@@ -634,11 +635,12 @@ public class HiveGiraphRunner implements Tool {
    * @param giraphConf GiraphConfiguration
    */
   private void logOptions(GiraphConfiguration giraphConf) {
-    GiraphClasses<?, ?, ?, ?> classes = new GiraphClasses(giraphConf);
+    GiraphClasses<?, ?, ?> classes = new GiraphClasses(giraphConf);
 
     LOG.info(getClass().getSimpleName() + " with");
 
-    LOG.info(LOG_PREFIX + "-vertexClass=" + vertexClass.getCanonicalName());
+    LOG.info(LOG_PREFIX + "-computationClass=" +
+        computationClass.getCanonicalName());
 
     for (VertexInputFormatDescription description : vertexInputDescriptions) {
       LOG.info(LOG_PREFIX + "Vertex input: " + description);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/common/DefaultConfigurableAndTableSchemaAware.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/common/DefaultConfigurableAndTableSchemaAware.java b/giraph-hive/src/main/java/org/apache/giraph/hive/common/DefaultConfigurableAndTableSchemaAware.java
index 4ba69c9..cfc696c 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/common/DefaultConfigurableAndTableSchemaAware.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/common/DefaultConfigurableAndTableSchemaAware.java
@@ -32,12 +32,10 @@ import com.facebook.hiveio.schema.HiveTableSchemaAware;
  * @param <I> Vertex ID
  * @param <V> Vertex Value
  * @param <E> Edge Value
- * @param <M> Message Value
  */
 public class DefaultConfigurableAndTableSchemaAware<
-    I extends WritableComparable, V extends Writable, E extends Writable,
-    M extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, M>
+    I extends WritableComparable, V extends Writable, E extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
     implements HiveTableSchemaAware {
   /** Schema stored here */
   private HiveTableSchema tableSchema;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
index 73fc553..a3c323e 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
@@ -30,10 +30,10 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public abstract class AbstractHiveToEdge<I extends WritableComparable,
     E extends Writable>
-    extends DefaultConfigurableAndTableSchemaAware<I, Writable, E, Writable>
+    extends DefaultConfigurableAndTableSchemaAware<I, Writable, E>
     implements HiveToEdge<I, E> {
   @Override
-  public ImmutableClassesGiraphConfiguration<I, Writable, E, Writable>
+  public ImmutableClassesGiraphConfiguration<I, Writable, E>
   getConf() {
     return super.getConf();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeInputFormat.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeInputFormat.java
index f35b6ea..a0e9cf3 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeInputFormat.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeInputFormat.java
@@ -58,7 +58,7 @@ public class HiveEdgeInputFormat<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
     super.setConf(conf);
     HiveUtils.initializeHiveInput(
         hiveInputFormat,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeReader.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeReader.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeReader.java
index ba99267..a010307 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeReader.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/HiveEdgeReader.java
@@ -43,7 +43,7 @@ import static org.apache.giraph.hive.common.GiraphHiveConstants.HIVE_EDGE_INPUT;
  * @param <E> Edge Value
  */
 public class HiveEdgeReader<I extends WritableComparable, E extends Writable>
-    extends DefaultConfigurableAndTableSchemaAware<I, Writable, E, Writable>
+    extends DefaultConfigurableAndTableSchemaAware<I, Writable, E>
     implements GiraphReader<EdgeWithSource<I, E>> {
   /** Underlying Hive RecordReader used */
   private RecordReader<WritableComparable, HiveReadableRecord> hiveRecordReader;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertex.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertex.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertex.java
index b08a09f..45cc209 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertex.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertex.java
@@ -32,10 +32,10 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public abstract class AbstractHiveToVertex<I extends WritableComparable,
     V extends Writable, E extends Writable>
-    extends DefaultConfigurableAndTableSchemaAware<I, V, E, Writable>
+    extends DefaultConfigurableAndTableSchemaAware<I, V, E>
     implements HiveToVertex<I, V, E> {
   @Override
-  public ImmutableClassesGiraphConfiguration<I, V, E, Writable> getConf() {
+  public ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
     return super.getConf();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveToVertex.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveToVertex.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveToVertex.java
index c670876..a9736c6 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveToVertex.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveToVertex.java
@@ -39,7 +39,7 @@ import java.util.Iterator;
  */
 public interface HiveToVertex<I extends WritableComparable,
     V extends Writable, E extends Writable> extends
-    Iterator<Vertex<I, V, E, ?>> {
+    Iterator<Vertex<I, V, E>> {
   /**
    * Set the records which contain vertex input data
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexInputFormat.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexInputFormat.java
index 824d8a6..063c472 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexInputFormat.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexInputFormat.java
@@ -59,7 +59,7 @@ public class HiveVertexInputFormat<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     super.setConf(conf);
     HiveUtils.initializeHiveInput(
         hiveInputFormat,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexReader.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexReader.java
index 719d8ad..db61aff 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexReader.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/HiveVertexReader.java
@@ -45,8 +45,8 @@ import static org.apache.giraph.hive.common.GiraphHiveConstants.HIVE_VERTEX_INPU
  */
 public class HiveVertexReader<I extends WritableComparable,
     V extends Writable, E extends Writable>
-    extends DefaultConfigurableAndTableSchemaAware<I, V, E, Writable>
-    implements GiraphReader<Vertex<I, V, E, ?>> {
+    extends DefaultConfigurableAndTableSchemaAware<I, V, E>
+    implements GiraphReader<Vertex<I, V, E>> {
   /** Underlying Hive RecordReader used */
   private RecordReader<WritableComparable, HiveReadableRecord> hiveRecordReader;
 
@@ -103,7 +103,7 @@ public class HiveVertexReader<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, ?> next() {
+  public Vertex<I, V, E> next() {
     return hiveToVertex.next();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleHiveToVertex.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleHiveToVertex.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleHiveToVertex.java
index 6df269d..5483a68 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleHiveToVertex.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleHiveToVertex.java
@@ -43,7 +43,7 @@ public abstract class SimpleHiveToVertex<I extends WritableComparable,
   private Iterator<HiveReadableRecord> records;
 
   /** Reusable vertex object */
-  private Vertex<I, V, E, ?> reusableVertex = null;
+  private Vertex<I, V, E> reusableVertex = null;
 
   /**
    * Read the Vertex's ID from the HiveRecord given.
@@ -71,7 +71,7 @@ public abstract class SimpleHiveToVertex<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     super.setConf(conf);
     if (conf.reuseVertexObjects()) {
       reusableVertex = getConf().createVertex();
@@ -89,12 +89,12 @@ public abstract class SimpleHiveToVertex<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, ?> next() {
+  public Vertex<I, V, E> next() {
     HiveReadableRecord record = records.next();
     I id = getVertexId(record);
     V value = getVertexValue(record);
     Iterable<Edge<I, E>> edges = getEdges(record);
-    Vertex<I, V, E, ?> vertex = reusableVertex;
+    Vertex<I, V, E> vertex = reusableVertex;
     if (vertex == null) {
       vertex = getConf().createVertex();
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java b/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
index 52fe93f..477ce6e 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
@@ -30,5 +30,5 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public abstract class AbstractVertexToHive<I extends WritableComparable,
     V extends Writable, E extends Writable>
-    extends DefaultConfigurableAndTableSchemaAware<I, V, E, Writable>
+    extends DefaultConfigurableAndTableSchemaAware<I, V, E>
     implements VertexToHive<I, V, E> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexOutputFormat.java b/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexOutputFormat.java
index 05441f4..6968eef 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexOutputFormat.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexOutputFormat.java
@@ -61,7 +61,7 @@ public class HiveVertexOutputFormat<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     super.setConf(conf);
     HiveUtils.initializeHiveOutput(
         hiveOutputFormat,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexWriter.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexWriter.java b/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexWriter.java
index 357cf89..352dee3 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexWriter.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/output/HiveVertexWriter.java
@@ -118,7 +118,7 @@ public class HiveVertexWriter<I extends WritableComparable, V extends Writable,
   }
 
   @Override
-  public void writeVertex(Vertex<I, V, E, ?> vertex)
+  public void writeVertex(Vertex<I, V, E> vertex)
     throws IOException, InterruptedException {
     vertexToHive.saveVertex(vertex, reusableRecord, this);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/output/SimpleVertexToHive.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/output/SimpleVertexToHive.java b/giraph-hive/src/main/java/org/apache/giraph/hive/output/SimpleVertexToHive.java
index fdfb363..c3fb6b6 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/output/SimpleVertexToHive.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/output/SimpleVertexToHive.java
@@ -45,12 +45,12 @@ public abstract class SimpleVertexToHive<I extends WritableComparable,
    * @param vertex Vertex to read from.
    * @param record HiveRecord to write to.
    */
-  public abstract void fillRecord(Vertex<I, V, E, ?> vertex,
+  public abstract void fillRecord(Vertex<I, V, E> vertex,
       HiveWritableRecord record);
 
   @Override
   public final void saveVertex(
-      Vertex<I, V, E, ?> vertex,
+      Vertex<I, V, E> vertex,
       HiveRecord reusableRecord,
       HiveRecordSaver recordSaver) throws IOException, InterruptedException {
     fillRecord(vertex, reusableRecord);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/output/VertexToHive.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/output/VertexToHive.java b/giraph-hive/src/main/java/org/apache/giraph/hive/output/VertexToHive.java
index 85e438c..28f987e 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/output/VertexToHive.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/output/VertexToHive.java
@@ -50,6 +50,6 @@ public interface VertexToHive<I extends WritableComparable, V extends Writable,
    * @param reusableRecord Record to use for writing data to it.
    * @param recordSaver Saver of records
    */
-  void saveVertex(Vertex<I, V, E, ?> vertex, HiveRecord reusableRecord,
+  void saveVertex(Vertex<I, V, E> vertex, HiveRecord reusableRecord,
       HiveRecordSaver recordSaver) throws IOException, InterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/main/java/org/apache/giraph/hive/output/examples/HiveOutputIntIntVertex.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/output/examples/HiveOutputIntIntVertex.java b/giraph-hive/src/main/java/org/apache/giraph/hive/output/examples/HiveOutputIntIntVertex.java
index 7955915..975d03d 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/output/examples/HiveOutputIntIntVertex.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/output/examples/HiveOutputIntIntVertex.java
@@ -30,7 +30,7 @@ import com.facebook.hiveio.record.HiveWritableRecord;
 public class HiveOutputIntIntVertex extends SimpleVertexToHive<IntWritable,
     IntWritable, NullWritable> {
   @Override public void fillRecord(
-      Vertex<IntWritable, IntWritable, NullWritable, ?> vertex,
+      Vertex<IntWritable, IntWritable, NullWritable> vertex,
       HiveWritableRecord record) {
     record.set(0, (long) vertex.getId().get());
     record.set(1, (long) vertex.getValue().get());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationCountEdges.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationCountEdges.java b/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationCountEdges.java
new file mode 100644
index 0000000..14ab478
--- /dev/null
+++ b/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationCountEdges.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.giraph.hive.computations;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import java.io.IOException;
+
+public class ComputationCountEdges extends BasicComputation<IntWritable,
+    IntWritable, NullWritable, NullWritable> {
+  @Override
+  public void compute(Vertex<IntWritable, IntWritable, NullWritable> vertex,
+      Iterable<NullWritable> messages) throws IOException {
+    vertex.setValue(new IntWritable(vertex.getNumEdges()));
+    vertex.voteToHalt();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationSumEdges.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationSumEdges.java b/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationSumEdges.java
new file mode 100644
index 0000000..613095e
--- /dev/null
+++ b/giraph-hive/src/test/java/org/apache/giraph/hive/computations/ComputationSumEdges.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.giraph.hive.computations;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import java.io.IOException;
+
+public class ComputationSumEdges extends BasicComputation<IntWritable,
+    DoubleWritable, DoubleWritable, NullWritable> {
+  @Override
+  public void compute(
+      Vertex<IntWritable, DoubleWritable, DoubleWritable> vertex,
+      Iterable<NullWritable> messages) throws IOException {
+    double sum = 0;
+    for (Edge<IntWritable, DoubleWritable> edge : vertex.getEdges()) {
+      sum += edge.getValue().get();
+    }
+    vertex.setValue(new DoubleWritable(sum));
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/computations/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/computations/package-info.java b/giraph-hive/src/test/java/org/apache/giraph/hive/computations/package-info.java
new file mode 100644
index 0000000..5569bda
--- /dev/null
+++ b/giraph-hive/src/test/java/org/apache/giraph/hive/computations/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Computation implementations for Hive tests.
+ */
+package org.apache.giraph.hive.computations;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveEdgeInputTest.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveEdgeInputTest.java b/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveEdgeInputTest.java
index 2ad87fa..0bb083c 100644
--- a/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveEdgeInputTest.java
+++ b/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveEdgeInputTest.java
@@ -19,11 +19,11 @@ package org.apache.giraph.hive.input;
 
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.hive.Helpers;
+import org.apache.giraph.hive.computations.ComputationCountEdges;
+import org.apache.giraph.hive.computations.ComputationSumEdges;
 import org.apache.giraph.hive.input.edge.HiveEdgeInputFormat;
 import org.apache.giraph.hive.input.edge.examples.HiveIntDoubleEdge;
 import org.apache.giraph.hive.input.edge.examples.HiveIntNullEdge;
-import org.apache.giraph.hive.vertexes.VertexCountEdges;
-import org.apache.giraph.hive.vertexes.VertexSumEdges;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.thrift.TException;
@@ -71,7 +71,7 @@ public class HiveEdgeInputTest {
     GiraphConfiguration conf = new GiraphConfiguration();
     HIVE_EDGE_INPUT.setTable(conf, tableName);
     HIVE_EDGE_INPUT.setClass(conf, HiveIntNullEdge.class);
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setEdgeInputFormatClass(HiveEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> output = InternalVertexRunner.run(conf, new String[0], new String[0]);
@@ -103,7 +103,7 @@ public class HiveEdgeInputTest {
     HIVE_EDGE_INPUT.setTable(conf, tableName);
     HIVE_EDGE_INPUT.setPartition(conf, partition);
     HIVE_EDGE_INPUT.setClass(conf, HiveIntNullEdge.class);
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setEdgeInputFormatClass(HiveEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> output = InternalVertexRunner.run(conf, new String[0], new String[0]);
@@ -134,7 +134,7 @@ public class HiveEdgeInputTest {
     GiraphConfiguration conf = new GiraphConfiguration();
     HIVE_EDGE_INPUT.setTable(conf, tableName);
     HIVE_EDGE_INPUT.setClass(conf, HiveIntDoubleEdge.class);
-    conf.setVertexClass(VertexSumEdges.class);
+    conf.setComputationClass(ComputationSumEdges.class);
     conf.setEdgeInputFormatClass(HiveEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> output = InternalVertexRunner.run(conf, new String[0], new String[0]);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveVertexInputTest.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveVertexInputTest.java b/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveVertexInputTest.java
index b6ab139..ec2b7b1 100644
--- a/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveVertexInputTest.java
+++ b/giraph-hive/src/test/java/org/apache/giraph/hive/input/HiveVertexInputTest.java
@@ -19,11 +19,11 @@ package org.apache.giraph.hive.input;
 
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.hive.Helpers;
+import org.apache.giraph.hive.computations.ComputationCountEdges;
+import org.apache.giraph.hive.computations.ComputationSumEdges;
 import org.apache.giraph.hive.input.vertex.HiveVertexInputFormat;
 import org.apache.giraph.hive.input.vertex.examples.HiveIntDoubleDoubleVertex;
 import org.apache.giraph.hive.input.vertex.examples.HiveIntNullNullVertex;
-import org.apache.giraph.hive.vertexes.VertexCountEdges;
-import org.apache.giraph.hive.vertexes.VertexSumEdges;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.thrift.TException;
@@ -36,8 +36,6 @@ import com.facebook.hiveio.testing.LocalHiveServer;
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 import static junit.framework.Assert.assertEquals;
 import static org.apache.giraph.hive.common.GiraphHiveConstants.HIVE_VERTEX_INPUT;
@@ -74,7 +72,7 @@ public class HiveVertexInputTest {
     GiraphConfiguration conf = new GiraphConfiguration();
     HIVE_VERTEX_INPUT.setTable(conf, tableName);
     HIVE_VERTEX_INPUT.setClass(conf, HiveIntNullNullVertex.class);
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setVertexInputFormatClass(HiveVertexInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> output = InternalVertexRunner.run(conf, new String[0], new String[0]);
@@ -107,7 +105,7 @@ public class HiveVertexInputTest {
     HIVE_VERTEX_INPUT.setTable(conf, tableName);
     HIVE_VERTEX_INPUT.setPartition(conf, partition);
     HIVE_VERTEX_INPUT.setClass(conf, HiveIntNullNullVertex.class);
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setVertexInputFormatClass(HiveVertexInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> output = InternalVertexRunner.run(conf, new String[0], new String[0]);
@@ -138,7 +136,7 @@ public class HiveVertexInputTest {
     GiraphConfiguration conf = new GiraphConfiguration();
     HIVE_VERTEX_INPUT.setTable(conf, tableName);
     HIVE_VERTEX_INPUT.setClass(conf, HiveIntDoubleDoubleVertex.class);
-    conf.setVertexClass(VertexSumEdges.class);
+    conf.setComputationClass(ComputationSumEdges.class);
     conf.setVertexInputFormatClass(HiveVertexInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> output = InternalVertexRunner.run(conf, new String[0], new String[0]);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/output/HiveOutputTest.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/output/HiveOutputTest.java b/giraph-hive/src/test/java/org/apache/giraph/hive/output/HiveOutputTest.java
index 673c9d5..43e853b 100644
--- a/giraph-hive/src/test/java/org/apache/giraph/hive/output/HiveOutputTest.java
+++ b/giraph-hive/src/test/java/org/apache/giraph/hive/output/HiveOutputTest.java
@@ -22,8 +22,8 @@ import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.hive.Helpers;
 import org.apache.giraph.hive.common.GiraphHiveConstants;
+import org.apache.giraph.hive.computations.ComputationCountEdges;
 import org.apache.giraph.hive.output.examples.HiveOutputIntIntVertex;
-import org.apache.giraph.hive.vertexes.VertexCountEdges;
 import org.apache.giraph.io.formats.IntNullTextEdgeInputFormat;
 import org.apache.giraph.io.internal.WrappedVertexOutputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
@@ -112,7 +112,7 @@ public class HiveOutputTest {
     GiraphHiveConstants.HIVE_VERTEX_OUTPUT_TABLE.set(conf, tableName);
     GiraphHiveConstants.VERTEX_TO_HIVE_CLASS.set(conf, HiveOutputIntIntVertex.class);
 
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(HiveVertexOutputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexCountEdges.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexCountEdges.java b/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexCountEdges.java
deleted file mode 100644
index a6db2af..0000000
--- a/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexCountEdges.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.giraph.hive.vertexes;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-
-import java.io.IOException;
-
-public class VertexCountEdges extends Vertex<IntWritable, IntWritable,
-    NullWritable, NullWritable> {
-  @Override public void compute(Iterable<NullWritable> messages)
-      throws IOException {
-    setValue(new IntWritable(getNumEdges()));
-    voteToHalt();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexSumEdges.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexSumEdges.java b/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexSumEdges.java
deleted file mode 100644
index d8527b4..0000000
--- a/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/VertexSumEdges.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.giraph.hive.vertexes;
-
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-
-import java.io.IOException;
-
-public class VertexSumEdges extends Vertex<IntWritable, DoubleWritable,
-    DoubleWritable, NullWritable> {
-  @Override public void compute(Iterable<NullWritable> messages)
-      throws IOException {
-    double sum = 0;
-    for (Edge<IntWritable, DoubleWritable> edge : getEdges()) {
-      sum += edge.getValue().get();
-    }
-    setValue(new DoubleWritable(sum));
-    voteToHalt();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/package-info.java b/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/package-info.java
deleted file mode 100644
index 552bb91..0000000
--- a/giraph-hive/src/test/java/org/apache/giraph/hive/vertexes/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * Vertex implementations for Hive tests.
- */
-package org.apache.giraph.hive.vertexes;


[05/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java b/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
index 0039ad6..d210928 100644
--- a/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
@@ -22,12 +22,13 @@ import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.WorkerClientRequestProcessor;
 import org.apache.giraph.comm.messages.ByteArrayMessagesPerVertexStore;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.ArrayListEdges;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.partition.BasicPartitionOwner;
 import org.apache.giraph.partition.PartitionOwner;
-import org.apache.giraph.edge.ArrayListEdges;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
@@ -55,7 +56,7 @@ public class MockUtils {
     public static class MockedEnvironment<I extends WritableComparable,
             V extends Writable, E extends Writable, M extends Writable> {
 
-        private final GraphState<I, V, E, M> graphState;
+        private final GraphState graphState;
         private final Mapper.Context context;
         private final Configuration conf;
         private final WorkerClientRequestProcessor workerClientRequestProcessor;
@@ -101,14 +102,15 @@ public class MockUtils {
     }
 
     /**
-     * prepare a vertex for use in a unit test by setting its internal state and injecting mocked
-     * dependencies,
+     * prepare a vertex and computation for use in a unit test by setting its
+     * internal state and injecting mocked dependencies,
      *
-     * @param vertex
-     * @param superstep the superstep to emulate
+     * @param vertex Vertex
      * @param vertexId initial vertex id
      * @param vertexValue initial vertex value
      * @param isHalted initial halted state of the vertex
+     * @param computation Computation
+     * @param superstep Superstep
      * @param <I> vertex id
      * @param <V> vertex data
      * @param <E> edge data
@@ -116,47 +118,42 @@ public class MockUtils {
      * @return
      * @throws Exception
      */
-    public static <I extends WritableComparable, V extends Writable,
-            E extends Writable, M extends Writable>
-            MockedEnvironment<I, V, E, M> prepareVertex(
-            Vertex<I, V, E, M> vertex, long superstep, I vertexId,
-            V vertexValue, boolean isHalted) throws Exception {
-
-        MockedEnvironment<I, V, E, M>  env =
-                new MockedEnvironment<I, V, E, M>();
-
-        Mockito.when(env.getGraphState().getSuperstep()).thenReturn(superstep);
-        Mockito.when(env.getGraphState().getContext())
-                .thenReturn(env.getContext());
-        Mockito.when(env.getContext().getConfiguration())
-                .thenReturn(env.getConfiguration());
-        Mockito.when(env.getGraphState().getWorkerClientRequestProcessor())
-                .thenReturn(env.getWorkerClientRequestProcessor());
-
-        GiraphConfiguration giraphConf = new GiraphConfiguration();
-        giraphConf.setVertexClass(vertex.getClass());
-        ImmutableClassesGiraphConfiguration<I, V, E, M> conf =
-            new ImmutableClassesGiraphConfiguration<I, V, E, M>(giraphConf);
-        vertex.setConf(conf);
-        ArrayListEdges<I, E> edges = new ArrayListEdges<I, E>();
-        edges.setConf((ImmutableClassesGiraphConfiguration<I, Writable, E,
-            Writable>) conf);
-        edges.initialize();
-
-        ReflectionUtils.setField(vertex, "id", vertexId);
-        ReflectionUtils.setField(vertex, "value", vertexValue);
-        ReflectionUtils.setField(vertex, "edges", edges);
-        ReflectionUtils.setField(vertex, "graphState", env.getGraphState());
-        ReflectionUtils.setField(vertex, "halt", isHalted);
-
-        return env;
+  public static <I extends WritableComparable, V extends Writable,
+      E extends Writable, M extends Writable>
+  MockedEnvironment<I, V, E, M> prepareVertexAndComputation(
+      Vertex<I, V, E> vertex, I vertexId, V vertexValue, boolean isHalted,
+      Computation<I, V, E, M, M> computation, long superstep) throws
+      Exception {
+    MockedEnvironment<I, V, E, M> env = new MockedEnvironment<I, V, E, M>();
+    Mockito.when(env.getGraphState().getSuperstep()).thenReturn(superstep);
+    Mockito.when(env.getGraphState().getContext())
+        .thenReturn(env.getContext());
+    Mockito.when(env.getContext().getConfiguration())
+        .thenReturn(env.getConfiguration());
+    computation.initialize(env.getGraphState(),
+        env.getWorkerClientRequestProcessor(), null, null, null);
+
+    GiraphConfiguration giraphConf = new GiraphConfiguration();
+    giraphConf.setComputationClass(computation.getClass());
+    giraphConf.setOutEdgesClass(ArrayListEdges.class);
+    ImmutableClassesGiraphConfiguration<I, V, E> conf =
+        new ImmutableClassesGiraphConfiguration<I, V, E>(giraphConf);
+    computation.setConf(conf);
+
+    vertex.setConf(conf);
+    vertex.initialize(vertexId, vertexValue);
+    if (isHalted) {
+      vertex.voteToHalt();
     }
 
+    return env;
+  }
+
   public static CentralizedServiceWorker<IntWritable, IntWritable,
-      IntWritable, IntWritable> mockServiceGetVertexPartitionOwner(final int
+      IntWritable> mockServiceGetVertexPartitionOwner(final int
       numOfPartitions) {
-    CentralizedServiceWorker<IntWritable, IntWritable, IntWritable,
-        IntWritable> service = Mockito.mock(CentralizedServiceWorker.class);
+    CentralizedServiceWorker<IntWritable, IntWritable, IntWritable> service =
+        Mockito.mock(CentralizedServiceWorker.class);
     Answer<PartitionOwner> answer = new Answer<PartitionOwner>() {
       @Override
       public PartitionOwner answer(InvocationOnMock invocation) throws
@@ -170,10 +167,10 @@ public class MockUtils {
     return service;
   }
 
-  public static ServerData<IntWritable, IntWritable, IntWritable, IntWritable>
+  public static ServerData<IntWritable, IntWritable, IntWritable>
   createNewServerData(ImmutableClassesGiraphConfiguration conf,
       Mapper.Context context) {
-    return new ServerData<IntWritable, IntWritable, IntWritable, IntWritable>(
+    return new ServerData<IntWritable, IntWritable, IntWritable>(
         Mockito.mock(CentralizedServiceWorker.class),
         conf,
         ByteArrayMessagesPerVertexStore.newFactory(

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/utils/NoOpComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/NoOpComputation.java b/giraph-core/src/test/java/org/apache/giraph/utils/NoOpComputation.java
new file mode 100644
index 0000000..630888f
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/NoOpComputation.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.utils;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.IOException;
+
+/**
+ * Computation which does nothing, just halts, used for testing
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ * @param <M> Message data
+ */
+public class NoOpComputation<I extends WritableComparable, V extends Writable,
+    E extends Writable, M extends Writable>
+    extends BasicComputation<I, V, E, M> {
+  @Override
+  public void compute(Vertex<I, V, E> vertex,
+      Iterable<M> messages) throws IOException {
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/vertices/IntIntNullVertexDoNothing.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/vertices/IntIntNullVertexDoNothing.java b/giraph-core/src/test/java/org/apache/giraph/vertices/IntIntNullVertexDoNothing.java
deleted file mode 100644
index c98d580..0000000
--- a/giraph-core/src/test/java/org/apache/giraph/vertices/IntIntNullVertexDoNothing.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.giraph.vertices;
-
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-
-public class IntIntNullVertexDoNothing extends VertexDoNothing<IntWritable,
-    IntWritable, NullWritable, NullWritable> {
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/vertices/VertexCountEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/vertices/VertexCountEdges.java b/giraph-core/src/test/java/org/apache/giraph/vertices/VertexCountEdges.java
deleted file mode 100644
index 9060bc7..0000000
--- a/giraph-core/src/test/java/org/apache/giraph/vertices/VertexCountEdges.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.giraph.vertices;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-
-import java.io.IOException;
-
-public class VertexCountEdges extends Vertex<IntWritable, IntWritable,
-    NullWritable, NullWritable> {
-  @Override
-  public void compute(Iterable<NullWritable> messages) throws IOException {
-    setValue(new IntWritable(getNumEdges()));
-    voteToHalt();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/vertices/VertexDoNothing.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/vertices/VertexDoNothing.java b/giraph-core/src/test/java/org/apache/giraph/vertices/VertexDoNothing.java
deleted file mode 100644
index fac3fce..0000000
--- a/giraph-core/src/test/java/org/apache/giraph/vertices/VertexDoNothing.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.giraph.vertices;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-import java.io.IOException;
-
-public class VertexDoNothing<I extends WritableComparable, V extends Writable,
-    E extends Writable, M extends Writable> extends Vertex<I, V, E, M> {
-  @Override
-  public void compute(Iterable<M> messages) throws IOException {
-    voteToHalt();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/yarn/TestYarnJob.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/yarn/TestYarnJob.java b/giraph-core/src/test/java/org/apache/giraph/yarn/TestYarnJob.java
index f9d5544..ed365b4 100644
--- a/giraph-core/src/test/java/org/apache/giraph/yarn/TestYarnJob.java
+++ b/giraph-core/src/test/java/org/apache/giraph/yarn/TestYarnJob.java
@@ -30,6 +30,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.formats.GiraphFileInputFormat;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
@@ -61,11 +62,12 @@ public class TestYarnJob implements Watcher {
   /**
    * Simple No-Op vertex to test if we can run a quick Giraph job on YARN.
    */
-  private static class DummyYarnVertex extends Vertex<IntWritable, IntWritable,
-      NullWritable, IntWritable> {
+  private static class DummyYarnComputation extends BasicComputation<
+      IntWritable, IntWritable, NullWritable, IntWritable> {
     @Override
-    public void compute(Iterable<IntWritable> messages) throws IOException {
-      voteToHalt();
+    public void compute(Vertex<IntWritable, IntWritable, NullWritable> vertex,
+        Iterable<IntWritable> messages) throws IOException {
+      vertex.voteToHalt();
     }
   }
 
@@ -203,7 +205,7 @@ public class TestYarnJob implements Watcher {
     conf.setEventWaitMsecs(3 * 1000);
     conf.setYarnLibJars(""); // no need
     conf.setYarnTaskHeapMb(256); // small since no work to be done
-    conf.setVertexClass(DummyYarnVertex.class);
+    conf.setComputationClass(DummyYarnComputation.class);
     conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     conf.setNumComputeThreads(1);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestComputation.java
new file mode 100644
index 0000000..db527f2
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestComputation.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+
+import java.io.IOException;
+
+/** Computation which uses aggrergators. To be used for testing. */
+public class AggregatorsTestComputation extends
+    BasicComputation<LongWritable, DoubleWritable, FloatWritable,
+        DoubleWritable> {
+
+  /** Name of regular aggregator */
+  private static final String REGULAR_AGG = "regular";
+  /** Name of persistent aggregator */
+  private static final String PERSISTENT_AGG = "persistent";
+  /** Name of master overwriting aggregator */
+  private static final String MASTER_WRITE_AGG = "master";
+  /** Value which master compute will use */
+  private static final long MASTER_VALUE = 12345;
+  /** Prefix for name of aggregators in array */
+  private static final String ARRAY_PREFIX_AGG = "array";
+  /** Number of aggregators to use in array */
+  private static final int NUM_OF_AGGREGATORS_IN_ARRAY = 100;
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    long superstep = getSuperstep();
+
+    LongWritable myValue = new LongWritable(1L << superstep);
+    aggregate(REGULAR_AGG, myValue);
+    aggregate(PERSISTENT_AGG, myValue);
+
+    long nv = getTotalNumVertices();
+    if (superstep > 0) {
+      assertEquals(nv * (1L << (superstep - 1)),
+          ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
+    } else {
+      assertEquals(0,
+          ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
+    }
+    assertEquals(nv * ((1L << superstep) - 1),
+        ((LongWritable) getAggregatedValue(PERSISTENT_AGG)).get());
+    assertEquals(MASTER_VALUE * (1L << superstep),
+        ((LongWritable) getAggregatedValue(MASTER_WRITE_AGG)).get());
+
+    for (int i = 0; i < NUM_OF_AGGREGATORS_IN_ARRAY; i++) {
+      aggregate(ARRAY_PREFIX_AGG + i, new LongWritable((superstep + 1) * i));
+      assertEquals(superstep * getTotalNumVertices() * i,
+          ((LongWritable) getAggregatedValue(ARRAY_PREFIX_AGG + i)).get());
+    }
+
+    if (getSuperstep() == 10) {
+      vertex.voteToHalt();
+    }
+  }
+
+  /** Master compute which uses aggregators. To be used for testing. */
+  public static class AggregatorsTestMasterCompute extends
+      DefaultMasterCompute {
+    @Override
+    public void compute() {
+      long superstep = getSuperstep();
+
+      LongWritable myValue =
+          new LongWritable(MASTER_VALUE * (1L << superstep));
+      setAggregatedValue(MASTER_WRITE_AGG, myValue);
+
+      long nv = getTotalNumVertices();
+      if (superstep > 0) {
+        assertEquals(nv * (1L << (superstep - 1)),
+            ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
+      } else {
+        assertEquals(0,
+            ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
+      }
+      assertEquals(nv * ((1L << superstep) - 1),
+          ((LongWritable) getAggregatedValue(PERSISTENT_AGG)).get());
+
+      for (int i = 0; i < NUM_OF_AGGREGATORS_IN_ARRAY; i++) {
+        assertEquals(superstep * getTotalNumVertices() * i,
+            ((LongWritable) getAggregatedValue(ARRAY_PREFIX_AGG + i)).get());
+      }
+    }
+
+    @Override
+    public void initialize() throws InstantiationException,
+        IllegalAccessException {
+      registerAggregator(REGULAR_AGG, LongSumAggregator.class);
+      registerPersistentAggregator(PERSISTENT_AGG,
+          LongSumAggregator.class);
+      registerAggregator(MASTER_WRITE_AGG, LongSumAggregator.class);
+
+      for (int i = 0; i < NUM_OF_AGGREGATORS_IN_ARRAY; i++) {
+        registerAggregator(ARRAY_PREFIX_AGG + i, LongSumAggregator.class);
+      }
+    }
+  }
+
+  /**
+   * Throws exception if values are not equal.
+   *
+   * @param expected Expected value
+   * @param actual   Actual value
+   */
+  private static void assertEquals(long expected, long actual) {
+    if (expected != actual) {
+      throw new RuntimeException("expected: " + expected +
+          ", actual: " + actual);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java
deleted file mode 100644
index d08519b..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/AggregatorsTestVertex.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.master.DefaultMasterCompute;
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-
-import java.io.IOException;
-
-/** Vertex which uses aggrergators. To be used for testing. */
-public class AggregatorsTestVertex extends
-    Vertex<LongWritable, DoubleWritable, FloatWritable,
-        DoubleWritable> {
-
-  /** Name of regular aggregator */
-  private static final String REGULAR_AGG = "regular";
-  /** Name of persistent aggregator */
-  private static final String PERSISTENT_AGG = "persistent";
-  /** Name of master overwriting aggregator */
-  private static final String MASTER_WRITE_AGG = "master";
-  /** Value which master compute will use */
-  private static final long MASTER_VALUE = 12345;
-  /** Prefix for name of aggregators in array */
-  private static final String ARRAY_PREFIX_AGG = "array";
-  /** Number of aggregators to use in array */
-  private static final int NUM_OF_AGGREGATORS_IN_ARRAY = 100;
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) throws IOException {
-    long superstep = getSuperstep();
-
-    LongWritable myValue = new LongWritable(1L << superstep);
-    aggregate(REGULAR_AGG, myValue);
-    aggregate(PERSISTENT_AGG, myValue);
-
-    long nv = getTotalNumVertices();
-    if (superstep > 0) {
-      assertEquals(nv * (1L << (superstep - 1)),
-          ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
-    } else {
-      assertEquals(0,
-          ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
-    }
-    assertEquals(nv * ((1L << superstep) - 1),
-        ((LongWritable) getAggregatedValue(PERSISTENT_AGG)).get());
-    assertEquals(MASTER_VALUE * (1L << superstep),
-        ((LongWritable) getAggregatedValue(MASTER_WRITE_AGG)).get());
-
-    for (int i = 0; i < NUM_OF_AGGREGATORS_IN_ARRAY; i++) {
-      aggregate(ARRAY_PREFIX_AGG + i, new LongWritable((superstep + 1) * i));
-      assertEquals(superstep * getTotalNumVertices() * i,
-          ((LongWritable) getAggregatedValue(ARRAY_PREFIX_AGG + i)).get());
-    }
-
-    if (getSuperstep() == 10) {
-      voteToHalt();
-    }
-  }
-
-  /** Master compute which uses aggregators. To be used for testing. */
-  public static class AggregatorsTestMasterCompute extends
-      DefaultMasterCompute {
-    @Override
-    public void compute() {
-      long superstep = getSuperstep();
-
-      LongWritable myValue =
-          new LongWritable(MASTER_VALUE * (1L << superstep));
-      setAggregatedValue(MASTER_WRITE_AGG, myValue);
-
-      long nv = getTotalNumVertices();
-      if (superstep > 0) {
-        assertEquals(nv * (1L << (superstep - 1)),
-            ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
-      } else {
-        assertEquals(0,
-            ((LongWritable) getAggregatedValue(REGULAR_AGG)).get());
-      }
-      assertEquals(nv * ((1L << superstep) - 1),
-          ((LongWritable) getAggregatedValue(PERSISTENT_AGG)).get());
-
-      for (int i = 0; i < NUM_OF_AGGREGATORS_IN_ARRAY; i++) {
-        assertEquals(superstep * getTotalNumVertices() * i,
-            ((LongWritable) getAggregatedValue(ARRAY_PREFIX_AGG + i)).get());
-      }
-    }
-
-    @Override
-    public void initialize() throws InstantiationException,
-        IllegalAccessException {
-      registerAggregator(REGULAR_AGG, LongSumAggregator.class);
-      registerPersistentAggregator(PERSISTENT_AGG,
-          LongSumAggregator.class);
-      registerAggregator(MASTER_WRITE_AGG, LongSumAggregator.class);
-
-      for (int i = 0; i < NUM_OF_AGGREGATORS_IN_ARRAY; i++) {
-        registerAggregator(ARRAY_PREFIX_AGG + i, LongSumAggregator.class);
-      }
-    }
-  }
-
-  /**
-   * Throws exception if values are not equal.
-   *
-   * @param expected Expected value
-   * @param actual   Actual value
-   */
-  private static void assertEquals(long expected, long actual) {
-    if (expected != actual) {
-      throw new RuntimeException("expected: " + expected +
-          ", actual: " + actual);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsComputation.java
new file mode 100644
index 0000000..9b0cfe1
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsComputation.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import java.io.IOException;
+
+/**
+ * Implementation of the HCC algorithm that identifies connected components and
+ * assigns each vertex its "component identifier" (the smallest vertex id
+ * in the component)
+ *
+ * The idea behind the algorithm is very simple: propagate the smallest
+ * vertex id along the edges to all vertices of a connected component. The
+ * number of supersteps necessary is equal to the length of the maximum
+ * diameter of all components + 1
+ *
+ * The original Hadoop-based variant of this algorithm was proposed by Kang,
+ * Charalampos, Tsourakakis and Faloutsos in
+ * "PEGASUS: Mining Peta-Scale Graphs", 2010
+ *
+ * http://www.cs.cmu.edu/~ukang/papers/PegasusKAIS.pdf
+ */
+@Algorithm(
+    name = "Connected components",
+    description = "Finds connected components of the graph"
+)
+public class ConnectedComponentsComputation extends
+    BasicComputation<IntWritable, IntWritable, NullWritable, IntWritable> {
+  /**
+   * Propagates the smallest vertex id to all neighbors. Will always choose to
+   * halt and only reactivate if a smaller id has been sent to it.
+   *
+   * @param vertex Vertex
+   * @param messages Iterator of messages from the previous superstep.
+   * @throws IOException
+   */
+  @Override
+  public void compute(
+      Vertex<IntWritable, IntWritable, NullWritable> vertex,
+      Iterable<IntWritable> messages) throws IOException {
+    int currentComponent = vertex.getValue().get();
+
+    // First superstep is special, because we can simply look at the neighbors
+    if (getSuperstep() == 0) {
+      for (Edge<IntWritable, NullWritable> edge : vertex.getEdges()) {
+        int neighbor = edge.getTargetVertexId().get();
+        if (neighbor < currentComponent) {
+          currentComponent = neighbor;
+        }
+      }
+      // Only need to send value if it is not the own id
+      if (currentComponent != vertex.getValue().get()) {
+        vertex.setValue(new IntWritable(currentComponent));
+        for (Edge<IntWritable, NullWritable> edge : vertex.getEdges()) {
+          IntWritable neighbor = edge.getTargetVertexId();
+          if (neighbor.get() > currentComponent) {
+            sendMessage(neighbor, vertex.getValue());
+          }
+        }
+      }
+
+      vertex.voteToHalt();
+      return;
+    }
+
+    boolean changed = false;
+    // did we get a smaller id ?
+    for (IntWritable message : messages) {
+      int candidateComponent = message.get();
+      if (candidateComponent < currentComponent) {
+        currentComponent = candidateComponent;
+        changed = true;
+      }
+    }
+
+    // propagate new component id to the neighbors
+    if (changed) {
+      vertex.setValue(new IntWritable(currentComponent));
+      sendMessageToAllEdges(vertex, vertex.getValue());
+    }
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java
deleted file mode 100644
index dbeb6bf..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/ConnectedComponentsVertex.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-
-import java.io.IOException;
-
-/**
- * Implementation of the HCC algorithm that identifies connected components and
- * assigns each vertex its "component identifier" (the smallest vertex id
- * in the component)
- *
- * The idea behind the algorithm is very simple: propagate the smallest
- * vertex id along the edges to all vertices of a connected component. The
- * number of supersteps necessary is equal to the length of the maximum
- * diameter of all components + 1
- *
- * The original Hadoop-based variant of this algorithm was proposed by Kang,
- * Charalampos, Tsourakakis and Faloutsos in
- * "PEGASUS: Mining Peta-Scale Graphs", 2010
- *
- * http://www.cs.cmu.edu/~ukang/papers/PegasusKAIS.pdf
- */
-@Algorithm(
-    name = "Connected components",
-    description = "Finds connected components of the graph"
-)
-public class ConnectedComponentsVertex extends Vertex<IntWritable,
-    IntWritable, NullWritable, IntWritable> {
-  /**
-   * Propagates the smallest vertex id to all neighbors. Will always choose to
-   * halt and only reactivate if a smaller id has been sent to it.
-   *
-   * @param messages Iterator of messages from the previous superstep.
-   * @throws IOException
-   */
-  @Override
-  public void compute(Iterable<IntWritable> messages) throws IOException {
-    int currentComponent = getValue().get();
-
-    // First superstep is special, because we can simply look at the neighbors
-    if (getSuperstep() == 0) {
-      for (Edge<IntWritable, NullWritable> edge : getEdges()) {
-        int neighbor = edge.getTargetVertexId().get();
-        if (neighbor < currentComponent) {
-          currentComponent = neighbor;
-        }
-      }
-      // Only need to send value if it is not the own id
-      if (currentComponent != getValue().get()) {
-        setValue(new IntWritable(currentComponent));
-        for (Edge<IntWritable, NullWritable> edge : getEdges()) {
-          IntWritable neighbor = edge.getTargetVertexId();
-          if (neighbor.get() > currentComponent) {
-            sendMessage(neighbor, getValue());
-          }
-        }
-      }
-
-      voteToHalt();
-      return;
-    }
-
-    boolean changed = false;
-    // did we get a smaller id ?
-    for (IntWritable message : messages) {
-      int candidateComponent = message.get();
-      if (candidateComponent < currentComponent) {
-        currentComponent = candidateComponent;
-        changed = true;
-      }
-    }
-
-    // propagate new component id to the neighbors
-    if (changed) {
-      setValue(new IntWritable(currentComponent));
-      sendMessageToAllEdges(getValue());
-    }
-    voteToHalt();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityComputation.java
new file mode 100644
index 0000000..12a58d5
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityComputation.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.io.IOException;
+
+/**
+ * User applications can subclass IdentityComputation, which
+ * simply prints the results that have been read for testing IO related
+ * jobs under any inputformat
+ *
+ * @param <I> Vertex index value
+ * @param <V> Vertex value
+ * @param <E> Edge value
+ * @param <M> Message value
+ */
+@SuppressWarnings("rawtypes")
+public class IdentityComputation<I extends WritableComparable,
+  V extends Writable, E extends Writable, M extends Writable>
+  extends BasicComputation<I, V, E, M> {
+  @Override
+  public void compute(Vertex<I, V, E> vertex,
+      Iterable<M> messages) throws IOException {
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityVertex.java
deleted file mode 100644
index 30cca86..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/IdentityVertex.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * User applications can subclass IdentityVertex, which
- * simply prints the results that have been read for testing IO related
- * jobs under any inputformat
- *
- * @param <I> Vertex index value
- * @param <V> Vertex value
- * @param <E> Edge value
- * @param <M> Message value
- */
-@SuppressWarnings("rawtypes")
-
-public abstract class IdentityVertex<I extends WritableComparable,
-  V extends Writable, E extends Writable, M extends Writable>
-  extends Vertex<I, V, E, M> {
-
-  @Override
-  public void compute(Iterable<M> messages) {
-    voteToHalt();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
index 62bea5a..f56b4f6 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
@@ -27,7 +27,6 @@ import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
@@ -44,10 +43,10 @@ public class LongDoubleDoubleTextInputFormat
     extends TextVertexInputFormat<LongWritable, DoubleWritable,
     DoubleWritable>
     implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
-    DoubleWritable, Writable> {
+    DoubleWritable> {
   /** Configuration. */
   private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, Writable> conf;
+      DoubleWritable> conf;
 
   @Override
   public TextVertexReader createVertexReader(InputSplit split,
@@ -58,13 +57,13 @@ public class LongDoubleDoubleTextInputFormat
 
   @Override
   public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
-      DoubleWritable, DoubleWritable, Writable> configuration) {
+      DoubleWritable, DoubleWritable> configuration) {
     this.conf = configuration;
   }
 
   @Override
   public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, Writable> getConf() {
+      DoubleWritable> getConf() {
     return conf;
   }
 
@@ -79,9 +78,9 @@ public class LongDoubleDoubleTextInputFormat
     private final Pattern separator = Pattern.compile("[\t ]");
 
     @Override
-    public Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+    public Vertex<LongWritable, DoubleWritable, DoubleWritable>
     getCurrentVertex() throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+      Vertex<LongWritable, DoubleWritable, DoubleWritable>
         vertex = conf.createVertex();
 
       String[] tokens =

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
index fdc9050..bfb5f40 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
@@ -28,7 +28,6 @@ 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.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
@@ -42,10 +41,10 @@ import java.util.regex.Pattern;
 public class LongDoubleNullTextInputFormat
     extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable>
     implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
-    NullWritable, Writable> {
+    NullWritable> {
   /** Configuration. */
   private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      NullWritable, Writable> conf;
+      NullWritable> conf;
 
   @Override
   public TextVertexReader createVertexReader(InputSplit split,
@@ -56,13 +55,13 @@ public class LongDoubleNullTextInputFormat
 
   @Override
   public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
-      DoubleWritable, NullWritable, Writable> configuration) {
+      DoubleWritable, NullWritable> configuration) {
     this.conf = configuration;
   }
 
   @Override
   public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      NullWritable, Writable> getConf() {
+      NullWritable> getConf() {
     return conf;
   }
 
@@ -77,9 +76,9 @@ public class LongDoubleNullTextInputFormat
     private final Pattern separator = Pattern.compile("[\t ]");
 
     @Override
-    public Vertex<LongWritable, DoubleWritable, NullWritable, ?>
+    public Vertex<LongWritable, DoubleWritable, NullWritable>
     getCurrentVertex() throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable, NullWritable, ?>
+      Vertex<LongWritable, DoubleWritable, NullWritable>
           vertex = conf.createVertex();
 
       String[] tokens =

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
index 7dc8475..5023a4e 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
@@ -27,7 +27,6 @@ import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
@@ -45,10 +44,10 @@ public class NormalizingLongDoubleDoubleTextInputFormat
     extends
     TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable>
     implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
-    DoubleWritable, Writable> {
+    DoubleWritable> {
   /** Configuration. */
   private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, Writable> conf;
+      DoubleWritable> conf;
 
   @Override
   public TextVertexReader createVertexReader(
@@ -58,13 +57,13 @@ public class NormalizingLongDoubleDoubleTextInputFormat
 
   @Override
   public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
-      DoubleWritable, DoubleWritable, Writable> configuration) {
+      DoubleWritable, DoubleWritable> configuration) {
     conf = configuration;
   }
 
   @Override
   public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, Writable> getConf() {
+      DoubleWritable> getConf() {
     return conf;
   }
 
@@ -81,10 +80,10 @@ public class NormalizingLongDoubleDoubleTextInputFormat
 
     @Override
     public Vertex<LongWritable, DoubleWritable,
-        DoubleWritable, ?> getCurrentVertex()
+        DoubleWritable> getCurrentVertex()
       throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable,
-      DoubleWritable, ?> vertex = conf.createVertex();
+      Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex =
+          conf.createVertex();
 
       String[] tokens = edgeSeparator.split(getRecordReader()
           .getCurrentValue().toString());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankComputation.java
new file mode 100644
index 0000000..9ac90d9
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankComputation.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.MathUtils;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+/**
+ * The PageRank algorithm, with uniform transition probabilities on the edges
+ * http://en.wikipedia.org/wiki/PageRank
+ */
+public class PageRankComputation extends RandomWalkComputation<NullWritable> {
+  @Override
+  protected double transitionProbability(
+      Vertex<LongWritable, DoubleWritable, NullWritable> vertex,
+      double stateProbability, Edge<LongWritable, NullWritable> edge) {
+    return stateProbability / vertex.getNumEdges();
+  }
+
+  @Override
+  protected double recompute(
+      Vertex<LongWritable, DoubleWritable, NullWritable> vertex,
+      Iterable<DoubleWritable> partialRanks, double teleportationProbability) {
+    // rank contribution from incident neighbors
+    double rankFromNeighbors = MathUtils.sum(partialRanks);
+    // rank contribution from dangling vertices
+    double danglingContribution =
+        getDanglingProbability() / getTotalNumVertices();
+
+    // recompute rank
+    return (1d - teleportationProbability) *
+        (rankFromNeighbors + danglingContribution) +
+        teleportationProbability / getTotalNumVertices();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankVertex.java
deleted file mode 100644
index 9678b31..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/PageRankVertex.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.utils.MathUtils;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-
-/**
- * The PageRank algorithm, with uniform transition probabilities on the edges
- * http://en.wikipedia.org/wiki/PageRank
- */
-public class PageRankVertex extends RandomWalkVertex<NullWritable> {
-
-  @Override
-  protected double transitionProbability(double stateProbability,
-      Edge<LongWritable, NullWritable> edge) {
-    return stateProbability / getNumEdges();
-  }
-
-  @Override
-  protected double recompute(Iterable<DoubleWritable> partialRanks,
-                             double teleportationProbability) {
-
-    // rank contribution from incident neighbors
-    double rankFromNeighbors = MathUtils.sum(partialRanks);
-    // rank contribution from dangling vertices
-    double danglingContribution =
-        getDanglingProbability() / getTotalNumVertices();
-
-    // recompute rank
-    return (1d - teleportationProbability) *
-        (rankFromNeighbors + danglingContribution) +
-        teleportationProbability / getTotalNumVertices();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/PartitionContextTestVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/PartitionContextTestVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/PartitionContextTestVertex.java
deleted file mode 100644
index f617d8e..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/PartitionContextTestVertex.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.partition.DefaultPartitionContext;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.worker.DefaultWorkerContext;
-import org.apache.giraph.worker.WorkerContext;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-
-import java.io.IOException;
-
-/**
- * Vertex to test the functionality of PartitionContext
- */
-public class PartitionContextTestVertex extends
-    Vertex<LongWritable, DoubleWritable, FloatWritable,
-        DoubleWritable> {
-  /** How many compute threads to use in the test */
-  public static final int NUM_COMPUTE_THREADS = 10;
-  /** How many vertices to create for the test */
-  public static final int NUM_VERTICES = 100;
-  /** How many partitions to have */
-  public static final int NUM_PARTITIONS = 25;
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) throws IOException {
-    TestPartitionContextPartitionContext partitionContext =
-        (TestPartitionContextPartitionContext) getPartitionContext();
-    partitionContext.counter++;
-    if (getSuperstep() > 5) {
-      voteToHalt();
-    }
-  }
-
-  /**
-   * PartitionContext for TestPartitionContext
-   */
-  public static class TestPartitionContextPartitionContext extends
-      DefaultPartitionContext {
-    /**
-     * The counter should hold the number of vertices in this partition,
-     * plus the current superstep
-     */
-    private long counter;
-
-    @Override
-    public void preSuperstep(WorkerContext workerContext) {
-      counter =
-          ((TestPartitionContextWorkerContext) workerContext).superstepCounter;
-    }
-
-    @Override
-    public void postSuperstep(WorkerContext workerContext) {
-      ((TestPartitionContextWorkerContext) workerContext).totalCounter +=
-          counter;
-    }
-  }
-
-  /**
-   * WorkerContext for TestPartitionContext
-   */
-  public static class TestPartitionContextWorkerContext extends
-      DefaultWorkerContext {
-    /** Current superstep */
-    private long superstepCounter;
-    /**
-     * This counter should hold the sum of PartitionContext's counters
-     */
-    private long totalCounter;
-
-    @Override
-    public void preSuperstep() {
-      superstepCounter = getSuperstep();
-      totalCounter = 0;
-    }
-
-    @Override
-    public void postSuperstep() {
-      assertEquals(totalCounter,
-          NUM_PARTITIONS * superstepCounter + getTotalNumVertices());
-    }
-  }
-
-  /**
-   * Throws exception if values are not equal.
-   *
-   * @param expected Expected value
-   * @param actual   Actual value
-   */
-  private static void assertEquals(long expected, long actual) {
-    if (expected != actual) {
-      throw new RuntimeException("expected: " + expected +
-          ", actual: " + actual);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkComputation.java
new file mode 100644
index 0000000..ed95aae
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkComputation.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+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.Writable;
+
+import java.io.IOException;
+
+/**
+ * Base class for executing a random walk on a graph
+ *
+ * @param <E> edge type
+ */
+public abstract class RandomWalkComputation<E extends Writable>
+    extends BasicComputation<LongWritable, DoubleWritable, E, DoubleWritable> {
+  /** Configuration parameter for the number of supersteps to execute */
+  static final String MAX_SUPERSTEPS = RandomWalkComputation.class.getName() +
+      ".maxSupersteps";
+  /** Configuration parameter for the teleportation probability */
+  static final String TELEPORTATION_PROBABILITY = RandomWalkComputation.class
+      .getName() + ".teleportationProbability";
+  /** Name of aggregator for the probability of dangling vertices */
+  static final String CUMULATIVE_DANGLING_PROBABILITY =
+      RandomWalkComputation.class.getName() + ".cumulativeDanglingProbability";
+  /** Name of aggregator for the probability of all vertices */
+  static final String CUMULATIVE_PROBABILITY = RandomWalkComputation.class
+      .getName() + ".cumulativeProbability";
+    /** Name of aggregator for the probability of dangling vertices */
+  static final String NUM_DANGLING_VERTICES = RandomWalkComputation.class
+      .getName() + ".numDanglingVertices";
+  /** Name of aggregator for the L1 norm of the probability difference, used
+   * for covergence detection */
+  static final String L1_NORM_OF_PROBABILITY_DIFFERENCE =
+      RandomWalkComputation.class.getName() + ".l1NormOfProbabilityDifference";
+  /** Reusable {@link DoubleWritable} instance to avoid object instantiation */
+  private final DoubleWritable doubleWritable = new DoubleWritable();
+  /** Reusable {@link LongWritable} for counting dangling vertices */
+  private final LongWritable one = new LongWritable(1);
+
+  /**
+   * Compute an initial probability value for the vertex. Per default,
+   * we start with a uniform distribution.
+   * @return The initial probability value.
+   */
+  protected double initialProbability() {
+    return 1.0 / getTotalNumVertices();
+  }
+
+  /**
+   * Compute the probability of transitioning to a neighbor vertex
+   * @param vertex Vertex
+   * @param stateProbability current steady state probability of the vertex
+   * @param edge edge to neighbor
+   * @return the probability of transitioning to a neighbor vertex
+   */
+  protected abstract double transitionProbability(
+      Vertex<LongWritable, DoubleWritable, E> vertex,
+      double stateProbability,
+      Edge<LongWritable, E> edge);
+
+  /**
+   * Perform a single step of a random walk computation.
+   * @param vertex Vertex
+   * @param messages Messages received in the previous step.
+   * @param teleportationProbability Probability of teleporting to another
+   *          vertex.
+   * @return The new probability distribution value.
+   */
+  protected abstract double recompute(
+      Vertex<LongWritable, DoubleWritable, E> vertex,
+      Iterable<DoubleWritable> messages,
+      double teleportationProbability);
+
+  /**
+   * Returns the cumulative probability from dangling vertices.
+   * @return The cumulative probability from dangling vertices.
+   */
+  protected double getDanglingProbability() {
+    return this.<DoubleWritable>getAggregatedValue(
+        RandomWalkComputation.CUMULATIVE_DANGLING_PROBABILITY).get();
+  }
+
+  /**
+   * Returns the cumulative probability from dangling vertices.
+   * @return The cumulative probability from dangling vertices.
+   */
+  protected double getPreviousCumulativeProbability() {
+    return this.<DoubleWritable>getAggregatedValue(
+        RandomWalkComputation.CUMULATIVE_PROBABILITY).get();
+  }
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, E> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    double stateProbability;
+
+    if (getSuperstep() > 0) {
+
+      double previousStateProbability = vertex.getValue().get();
+      stateProbability =
+          recompute(vertex, messages, teleportationProbability());
+
+      // Important: rescale for numerical stability
+      stateProbability /= getPreviousCumulativeProbability();
+
+      doubleWritable.set(Math.abs(stateProbability - previousStateProbability));
+      aggregate(L1_NORM_OF_PROBABILITY_DIFFERENCE, doubleWritable);
+
+    } else {
+      stateProbability = initialProbability();
+    }
+
+    vertex.getValue().set(stateProbability);
+
+    aggregate(CUMULATIVE_PROBABILITY, vertex.getValue());
+
+    // Compute dangling node contribution for next superstep
+    if (vertex.getNumEdges() == 0) {
+      aggregate(NUM_DANGLING_VERTICES, one);
+      aggregate(CUMULATIVE_DANGLING_PROBABILITY, vertex.getValue());
+    }
+
+    if (getSuperstep() < maxSupersteps()) {
+      for (Edge<LongWritable, E> edge : vertex.getEdges()) {
+        double transitionProbability =
+            transitionProbability(vertex, stateProbability, edge);
+        doubleWritable.set(transitionProbability);
+        sendMessage(edge.getTargetVertexId(), doubleWritable);
+      }
+    } else {
+      vertex.voteToHalt();
+    }
+  }
+
+  /**
+   * Reads the number of supersteps to execute from the configuration
+   * @return number of supersteps to execute
+   */
+  private int maxSupersteps() {
+    return ((RandomWalkWorkerContext) getWorkerContext()).getMaxSupersteps();
+  }
+
+  /**
+   * Reads the teleportation probability from the configuration
+   * @return teleportation probability
+   */
+  protected double teleportationProbability() {
+    return ((RandomWalkWorkerContext) getWorkerContext())
+        .getTeleportationProbability();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java
deleted file mode 100644
index 2d2c988..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertex.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-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.Writable;
-
-import java.io.IOException;
-
-/**
- * Base class for executing a random walk on a graph
- *
- * @param <E> edge type
- */
-public abstract class RandomWalkVertex<E extends Writable>
-    extends Vertex<LongWritable, DoubleWritable, E, DoubleWritable> {
-  /** Configuration parameter for the number of supersteps to execute */
-  static final String MAX_SUPERSTEPS = RandomWalkVertex.class.getName() +
-      ".maxSupersteps";
-  /** Configuration parameter for the teleportation probability */
-  static final String TELEPORTATION_PROBABILITY = RandomWalkVertex.class
-      .getName() + ".teleportationProbability";
-  /** Name of aggregator for the probability of dangling vertices */
-  static final String CUMULATIVE_DANGLING_PROBABILITY = RandomWalkVertex.class
-      .getName() + ".cumulativeDanglingProbability";
-  /** Name of aggregator for the probability of all vertices */
-  static final String CUMULATIVE_PROBABILITY = RandomWalkVertex.class
-      .getName() + ".cumulativeProbability";
-    /** Name of aggregator for the probability of dangling vertices */
-  static final String NUM_DANGLING_VERTICES = RandomWalkVertex.class
-      .getName() + ".numDanglingVertices";
-  /** Name of aggregator for the L1 norm of the probability difference, used
-   * for covergence detection */
-  static final String L1_NORM_OF_PROBABILITY_DIFFERENCE = RandomWalkVertex.class
-      .getName() + ".l1NormOfProbabilityDifference";
-  /** Reusable {@link DoubleWritable} instance to avoid object instantiation */
-  private final DoubleWritable doubleWritable = new DoubleWritable();
-  /** Reusable {@link LongWritable} for counting dangling vertices */
-  private final LongWritable one = new LongWritable(1);
-
-  /**
-   * Compute an initial probability value for the vertex. Per default,
-   * we start with a uniform distribution.
-   * @return The initial probability value.
-   */
-  protected double initialProbability() {
-    return 1.0 / getTotalNumVertices();
-  }
-
-  /**
-   * Compute the probability of transitioning to a neighbor vertex
-   * @param stateProbability current steady state probability of the vertex
-   * @param edge edge to neighbor
-   * @return the probability of transitioning to a neighbor vertex
-   */
-  protected abstract double transitionProbability(double stateProbability,
-      Edge<LongWritable, E> edge);
-
-  /**
-   * Perform a single step of a random walk computation.
-   * @param messages Messages received in the previous step.
-   * @param teleportationProbability Probability of teleporting to another
-   *          vertex.
-   * @return The new probability distribution value.
-   */
-  protected abstract double recompute(Iterable<DoubleWritable> messages,
-      double teleportationProbability);
-
-  /**
-   * Returns the cumulative probability from dangling vertices.
-   * @return The cumulative probability from dangling vertices.
-   */
-  protected double getDanglingProbability() {
-    return this.<DoubleWritable>getAggregatedValue(
-        RandomWalkVertex.CUMULATIVE_DANGLING_PROBABILITY).get();
-  }
-
-  /**
-   * Returns the cumulative probability from dangling vertices.
-   * @return The cumulative probability from dangling vertices.
-   */
-  protected double getPreviousCumulativeProbability() {
-    return this.<DoubleWritable>getAggregatedValue(
-        RandomWalkVertex.CUMULATIVE_PROBABILITY).get();
-  }
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) throws IOException {
-    double stateProbability;
-
-    if (getSuperstep() > 0) {
-
-      double previousStateProbability = getValue().get();
-      stateProbability = recompute(messages, teleportationProbability());
-
-      // Important: rescale for numerical stability
-      stateProbability /= getPreviousCumulativeProbability();
-
-      doubleWritable.set(Math.abs(stateProbability - previousStateProbability));
-      aggregate(L1_NORM_OF_PROBABILITY_DIFFERENCE, doubleWritable);
-
-    } else {
-      stateProbability = initialProbability();
-    }
-
-    getValue().set(stateProbability);
-
-    aggregate(CUMULATIVE_PROBABILITY, getValue());
-
-    // Compute dangling node contribution for next superstep
-    if (getNumEdges() == 0) {
-      aggregate(NUM_DANGLING_VERTICES, one);
-      aggregate(CUMULATIVE_DANGLING_PROBABILITY, getValue());
-    }
-
-    if (getSuperstep() < maxSupersteps()) {
-      for (Edge<LongWritable, E> edge : getEdges()) {
-        double transitionProbability =
-            transitionProbability(stateProbability, edge);
-        doubleWritable.set(transitionProbability);
-        sendMessage(edge.getTargetVertexId(), doubleWritable);
-      }
-    } else {
-      voteToHalt();
-    }
-  }
-
-  /**
-   * Reads the number of supersteps to execute from the configuration
-   * @return number of supersteps to execute
-   */
-  private int maxSupersteps() {
-    return ((RandomWalkWorkerContext) getWorkerContext()).getMaxSupersteps();
-  }
-
-  /**
-   * Reads the teleportation probability from the configuration
-   * @return teleportation probability
-   */
-  protected double teleportationProbability() {
-    return ((RandomWalkWorkerContext) getWorkerContext())
-        .getTeleportationProbability();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertexMasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertexMasterCompute.java b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertexMasterCompute.java
index 9e5dbbf..8b5f23b 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertexMasterCompute.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkVertexMasterCompute.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.log4j.Logger;
 
 /**
- * Master compute associated with {@link RandomWalkVertex}. It handles
+ * Master compute associated with {@link RandomWalkComputation}. It handles
  * dangling nodes.
  */
 public class RandomWalkVertexMasterCompute extends DefaultMasterCompute {
@@ -42,16 +42,16 @@ public class RandomWalkVertexMasterCompute extends DefaultMasterCompute {
   public void compute() {
     double danglingContribution =
         this.<DoubleWritable>getAggregatedValue(
-            RandomWalkVertex.CUMULATIVE_DANGLING_PROBABILITY).get();
+            RandomWalkComputation.CUMULATIVE_DANGLING_PROBABILITY).get();
     double cumulativeProbability =
         this.<DoubleWritable>getAggregatedValue(
-            RandomWalkVertex.CUMULATIVE_PROBABILITY).get();
+            RandomWalkComputation.CUMULATIVE_PROBABILITY).get();
     double l1NormOfStateDiff =
         this.<DoubleWritable>getAggregatedValue(
-            RandomWalkVertex.L1_NORM_OF_PROBABILITY_DIFFERENCE).get();
+            RandomWalkComputation.L1_NORM_OF_PROBABILITY_DIFFERENCE).get();
     long numDanglingVertices =
         this.<LongWritable>getAggregatedValue(
-            RandomWalkVertex.NUM_DANGLING_VERTICES).get();
+            RandomWalkComputation.NUM_DANGLING_VERTICES).get();
 
     LOG.info("[Superstep " + getSuperstep() + "] Dangling contribution = " +
         danglingContribution + ", number of dangling vertices = " +
@@ -69,13 +69,13 @@ public class RandomWalkVertexMasterCompute extends DefaultMasterCompute {
   @Override
   public void initialize() throws InstantiationException,
       IllegalAccessException {
-    registerAggregator(RandomWalkVertex.NUM_DANGLING_VERTICES,
+    registerAggregator(RandomWalkComputation.NUM_DANGLING_VERTICES,
         LongSumAggregator.class);
-    registerAggregator(RandomWalkVertex.CUMULATIVE_DANGLING_PROBABILITY,
+    registerAggregator(RandomWalkComputation.CUMULATIVE_DANGLING_PROBABILITY,
         DoubleSumAggregator.class);
-    registerAggregator(RandomWalkVertex.CUMULATIVE_PROBABILITY,
+    registerAggregator(RandomWalkComputation.CUMULATIVE_PROBABILITY,
         DoubleSumAggregator.class);
-    registerAggregator(RandomWalkVertex.L1_NORM_OF_PROBABILITY_DIFFERENCE,
+    registerAggregator(RandomWalkComputation.L1_NORM_OF_PROBABILITY_DIFFERENCE,
         DoubleSumAggregator.class);
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartComputation.java
new file mode 100644
index 0000000..94e5d60
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartComputation.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import com.google.common.base.Preconditions;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.MathUtils;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Executes "RandomWalkWithRestart", a random walk on the graph which is biased
+ * towards a source vertex. The resulting probabilities of staying at a given
+ * vertex can be interpreted as a measure of proximity to the source vertex.
+ */
+public class RandomWalkWithRestartComputation
+    extends RandomWalkComputation<DoubleWritable> {
+
+  /** Configuration parameter for the source vertex */
+  static final String SOURCE_VERTEX = RandomWalkWithRestartComputation.class
+      .getName() + ".sourceVertex";
+
+  /**
+   * Checks whether the currently executed vertex is the source vertex
+   * @param vertex Vertex
+   * @return is the currently executed vertex the source vertex?
+   */
+  private boolean isSourceVertex(Vertex<LongWritable, ?, ?> vertex) {
+    return ((RandomWalkWorkerContext) getWorkerContext()).isSource(
+        vertex.getId().get());
+  }
+
+  /**
+   * Returns the number of source vertices.
+   * @return The number of source vertices.
+   */
+  private int numSourceVertices() {
+    return ((RandomWalkWorkerContext) getWorkerContext()).numSources();
+  }
+
+  @Override
+  protected double transitionProbability(
+      Vertex<LongWritable, DoubleWritable, DoubleWritable>
+          vertex,
+      double stateProbability, Edge<LongWritable, DoubleWritable> edge) {
+    return stateProbability * edge.getValue().get();
+  }
+
+  @Override
+  protected double recompute(
+      Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex,
+      Iterable<DoubleWritable> transitionProbabilities,
+      double teleportationProbability) {
+    int numSourceVertices = numSourceVertices();
+    Preconditions.checkState(numSourceVertices > 0, "No source vertex found");
+
+    double stateProbability = MathUtils.sum(transitionProbabilities);
+    // Add the contribution of dangling nodes (weakly preferential
+    // implementation: dangling nodes redistribute uniformly)
+    stateProbability += getDanglingProbability() / getTotalNumVertices();
+    // The random walk might teleport back to one of the source vertexes
+    stateProbability *= 1 - teleportationProbability;
+    if (isSourceVertex(vertex)) {
+      stateProbability += teleportationProbability / numSourceVertices;
+    }
+    return stateProbability;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartVertex.java
deleted file mode 100644
index 6f3eb6c..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWithRestartVertex.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import com.google.common.base.Preconditions;
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.utils.MathUtils;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-
-/**
- * Executes "RandomWalkWithRestart", a random walk on the graph which is biased
- * towards a source vertex. The resulting probabilities of staying at a given
- * vertex can be interpreted as a measure of proximity to the source vertex.
- */
-public class RandomWalkWithRestartVertex
-    extends RandomWalkVertex<DoubleWritable> {
-
-  /** Configuration parameter for the source vertex */
-  static final String SOURCE_VERTEX = RandomWalkWithRestartVertex.class
-      .getName() + ".sourceVertex";
-
-  /**
-   * Checks whether the currently executed vertex is the source vertex
-   * @return is the currently executed vertex the source vertex?
-   */
-  private boolean isSourceVertex() {
-    return ((RandomWalkWorkerContext) getWorkerContext()).isSource(getId()
-        .get());
-  }
-
-  /**
-   * Returns the number of source vertices.
-   * @return The number of source vertices.
-   */
-  private int numSourceVertices() {
-    return ((RandomWalkWorkerContext) getWorkerContext()).numSources();
-  }
-
-  @Override
-  protected double transitionProbability(double stateProbability,
-      Edge<LongWritable, DoubleWritable> edge) {
-    return stateProbability * edge.getValue().get();
-  }
-
-  @Override
-  protected double recompute(Iterable<DoubleWritable> transitionProbabilities,
-      double teleportationProbability) {
-
-    int numSourceVertices = numSourceVertices();
-    Preconditions.checkState(numSourceVertices > 0, "No source vertex found");
-
-    double stateProbability = MathUtils.sum(transitionProbabilities);
-    // Add the contribution of dangling nodes (weakly preferential
-    // implementation: dangling nodes redistribute uniformly)
-    stateProbability += getDanglingProbability() / getTotalNumVertices();
-    // The random walk might teleport back to one of the source vertexes
-    stateProbability *= 1 - teleportationProbability;
-    if (isSourceVertex()) {
-      stateProbability += teleportationProbability / numSourceVertices;
-    }
-    return stateProbability;
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
index 2566f43..5c23b5a 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/RandomWalkWorkerContext.java
@@ -48,8 +48,8 @@ public class RandomWalkWorkerContext extends WorkerContext {
   private static Set<Long> SOURCES;
 
   /** Configuration parameter for the source vertex */
-  private static final String SOURCE_VERTEX = RandomWalkWithRestartVertex.class
-      .getName() + ".sourceVertex";
+  private static final String SOURCE_VERTEX =
+      RandomWalkWithRestartComputation.class.getName() + ".sourceVertex";
 
   /** Logger */
   private static final Logger LOG = Logger
@@ -143,10 +143,10 @@ public class RandomWalkWorkerContext extends WorkerContext {
   public void preApplication() throws InstantiationException,
       IllegalAccessException {
     Configuration configuration = getContext().getConfiguration();
-    MAX_SUPERSTEPS = configuration.getInt(RandomWalkVertex.MAX_SUPERSTEPS,
+    MAX_SUPERSTEPS = configuration.getInt(RandomWalkComputation.MAX_SUPERSTEPS,
         DEFAULT_MAX_SUPERSTEPS);
     TELEPORTATION_PROBABILITY = configuration.getFloat(
-        RandomWalkVertex.TELEPORTATION_PROBABILITY,
+        RandomWalkComputation.TELEPORTATION_PROBABILITY,
         DEFAULT_TELEPORTATION_PROBABILITY);
     SOURCES = initializeSources(configuration);
   }


[09/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java b/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
index 82e1b1e..99b28df 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java
@@ -43,7 +43,6 @@ import org.apache.giraph.utils.ProgressableUtils;
 import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.giraph.worker.BspServiceWorker;
 import org.apache.giraph.worker.InputSplitsCallable;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
 import org.apache.giraph.worker.WorkerContext;
 import org.apache.giraph.worker.WorkerObserver;
 import org.apache.giraph.zk.ZooKeeperManager;
@@ -72,9 +71,10 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
 import static org.apache.giraph.conf.GiraphConstants.EDGE_VALUE_CLASS;
-import static org.apache.giraph.conf.GiraphConstants.MESSAGE_VALUE_CLASS;
+import static org.apache.giraph.conf.GiraphConstants.INCOMING_MESSAGE_VALUE_CLASS;
 import static org.apache.giraph.conf.GiraphConstants.VERTEX_ID_CLASS;
 import static org.apache.giraph.conf.GiraphConstants.VERTEX_VALUE_CLASS;
+import static org.apache.giraph.conf.GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS;
 
 /**
  * The Giraph-specific business logic for a single BSP
@@ -86,11 +86,10 @@ import static org.apache.giraph.conf.GiraphConstants.VERTEX_VALUE_CLASS;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class GraphTaskManager<I extends WritableComparable, V extends Writable,
-  E extends Writable, M extends Writable> implements
+  E extends Writable> implements
   ResetSuperstepMetricsObserver {
   /*if_not[PURE_YARN]
   static { // Eliminate this? Even MRv1 tasks should not need it here.
@@ -112,9 +111,9 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(GraphTaskManager.class);
   /** Coordination service worker */
-  private CentralizedServiceWorker<I, V, E, M> serviceWorker;
+  private CentralizedServiceWorker<I, V, E> serviceWorker;
   /** Coordination service master */
-  private CentralizedServiceMaster<I, V, E, M> serviceMaster;
+  private CentralizedServiceMaster<I, V, E> serviceMaster;
   /** Coordination service master thread */
   private Thread masterThread = null;
   /** The worker should be run exactly once, or else there is a problem. */
@@ -122,7 +121,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   /** Manages the ZooKeeper servers if necessary (dynamic startup) */
   private ZooKeeperManager zkManager;
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Already complete? */
   private boolean done = false;
   /** What kind of functions is this mapper doing? */
@@ -176,7 +175,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   public void setup(Path[] zkPathList)
     throws IOException, InterruptedException {
     context.setStatus("setup: Beginning worker setup.");
-    conf = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
+    conf = new ImmutableClassesGiraphConfiguration<I, V, E>(
       context.getConfiguration());
     determineClassTypes(conf);
     // configure global logging level for Giraph job
@@ -233,8 +232,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
     if (collectInputSuperstepStats(finishedSuperstepStats)) {
       return;
     }
-    WorkerAggregatorUsage aggregatorUsage =
-      prepareAggregatorsAndGraphState();
+    prepareGraphStateAndWorkerContext();
     List<PartitionStats> partitionStatsList = new ArrayList<PartitionStats>();
     int numComputeThreads = conf.getNumComputeThreads();
 
@@ -243,24 +241,22 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
       final long superstep = serviceWorker.getSuperstep();
       GiraphTimerContext superstepTimerContext =
         getTimerForThisSuperstep(superstep);
-      GraphState<I, V, E, M> graphState =
-        new GraphState<I, V, E, M>(superstep,
-            finishedSuperstepStats.getVertexCount(),
-            finishedSuperstepStats.getEdgeCount(),
-          context, this, null, aggregatorUsage);
+      GraphState graphState = new GraphState(superstep,
+          finishedSuperstepStats.getVertexCount(),
+          finishedSuperstepStats.getEdgeCount(),
+          context);
       Collection<? extends PartitionOwner> masterAssignedPartitionOwners =
-        serviceWorker.startSuperstep(graphState);
+        serviceWorker.startSuperstep();
       if (LOG.isDebugEnabled()) {
         LOG.debug("execute: " + MemoryUtils.getRuntimeMemoryStats());
       }
       context.progress();
       serviceWorker.exchangeVertexPartitions(masterAssignedPartitionOwners);
       context.progress();
-      graphState = checkSuperstepRestarted(
-        aggregatorUsage, superstep, graphState);
+      graphState = checkSuperstepRestarted(superstep, graphState);
       prepareForSuperstep(graphState);
       context.progress();
-      MessageStoreByPartition<I, M> messageStore =
+      MessageStoreByPartition<I, Writable> messageStore =
         serviceWorker.getServerData().getCurrentMessageStore();
       int numPartitions = serviceWorker.getPartitionStore().getNumPartitions();
       int numThreads = Math.min(numComputeThreads, numPartitions);
@@ -276,14 +272,14 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
           messageStore, numPartitions, numThreads);
       }
       finishedSuperstepStats = completeSuperstepAndCollectStats(
-        partitionStatsList, superstepTimerContext, graphState);
+        partitionStatsList, superstepTimerContext);
       // END of superstep compute loop
     } while (!finishedSuperstepStats.allVerticesHalted());
 
     if (LOG.isInfoEnabled()) {
       LOG.info("execute: BSP application done (global vertices marked done)");
     }
-    updateSuperstepGraphState(aggregatorUsage);
+    updateSuperstepGraphState();
     postApplication();
   }
 
@@ -364,15 +360,12 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
 
   /**
    * Utility to place a new, updated GraphState object into the serviceWorker.
-   * @param aggregatorUsage handle to aggregation metadata
    */
-  private void updateSuperstepGraphState(
-    WorkerAggregatorUsage aggregatorUsage) {
+  private void updateSuperstepGraphState() {
     serviceWorker.getWorkerContext().setGraphState(
-      new GraphState<I, V, E, M>(serviceWorker.getSuperstep(),
-        finishedSuperstepStats.getVertexCount(),
-          finishedSuperstepStats.getEdgeCount(), context, this, null,
-          aggregatorUsage));
+        new GraphState(serviceWorker.getSuperstep(),
+            finishedSuperstepStats.getVertexCount(),
+            finishedSuperstepStats.getEdgeCount(), context));
   }
 
   /**
@@ -380,15 +373,12 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
    * end of each superstep processing loop in the <code>execute</code> method.
    * @param partitionStatsList list of stas for each superstep to append to
    * @param superstepTimerContext for job metrics
-   * @param graphState the graph state metadata
    * @return the collected stats at the close of the current superstep.
    */
   private FinishedSuperstepStats completeSuperstepAndCollectStats(
     List<PartitionStats> partitionStatsList,
-    GiraphTimerContext superstepTimerContext,
-    GraphState<I, V, E, M> graphState) {
-    finishedSuperstepStats =
-      serviceWorker.finishSuperstep(graphState, partitionStatsList);
+    GiraphTimerContext superstepTimerContext) {
+    finishedSuperstepStats = serviceWorker.finishSuperstep(partitionStatsList);
     superstepTimerContext.stop();
     if (conf.metricsEnabled()) {
       GiraphMetrics.get().perSuperstep().printSummary(System.err);
@@ -401,7 +391,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
    * operations for the next superstep.
    * @param graphState graph state metadata object
    */
-  private void prepareForSuperstep(GraphState<I, V, E, M> graphState) {
+  private void prepareForSuperstep(GraphState graphState) {
     serviceWorker.prepareSuperstep();
 
     serviceWorker.getWorkerContext().setGraphState(graphState);
@@ -417,15 +407,11 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   }
 
   /**
-   * Prepare aggregators and worker context for superstep cycles.
-   * @return aggregator metadata object
+   * Prepare graph state and worker context for superstep cycles.
    */
-  private WorkerAggregatorUsage prepareAggregatorsAndGraphState() {
-    WorkerAggregatorUsage aggregatorUsage =
-      serviceWorker.getAggregatorHandler();
-    updateSuperstepGraphState(aggregatorUsage);
+  private void prepareGraphStateAndWorkerContext() {
+    updateSuperstepGraphState();
     workerContextPreApp();
-    return aggregatorUsage;
   }
 
   /**
@@ -459,18 +445,22 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
   public void determineClassTypes(Configuration conf) {
     ImmutableClassesGiraphConfiguration giraphConf =
         new ImmutableClassesGiraphConfiguration(conf);
-    Class<? extends Vertex<I, V, E, M>> vertexClass =
-        giraphConf.getVertexClass();
-    List<Class<?>> classList = ReflectionUtils.<Vertex>getTypeArguments(
-        Vertex.class, vertexClass);
+    Class<? extends Computation<I, V, E, Writable, Writable>> computationClass =
+        giraphConf.getComputationClass();
+    List<Class<?>> classList = ReflectionUtils.<Computation>getTypeArguments(
+        Computation.class, computationClass);
     Type vertexIndexType = classList.get(0);
     Type vertexValueType = classList.get(1);
     Type edgeValueType = classList.get(2);
-    Type messageValueType = classList.get(3);
+    Type incomingMessageValueType = classList.get(3);
+    Type outgoingMessageValueType = classList.get(4);
     VERTEX_ID_CLASS.set(conf, (Class<WritableComparable>) vertexIndexType);
     VERTEX_VALUE_CLASS.set(conf, (Class<Writable>) vertexValueType);
     EDGE_VALUE_CLASS.set(conf, (Class<Writable>) edgeValueType);
-    MESSAGE_VALUE_CLASS.set(conf, (Class<Writable>) messageValueType);
+    INCOMING_MESSAGE_VALUE_CLASS.set(conf,
+        (Class<Writable>) incomingMessageValueType);
+    OUTGOING_MESSAGE_VALUE_CLASS.set(conf,
+        (Class<Writable>) outgoingMessageValueType);
   }
 
   /**
@@ -567,16 +557,16 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
         LOG.info("setup: Starting up BspServiceMaster " +
           "(master thread)...");
       }
-      serviceMaster = new BspServiceMaster<I, V, E, M>(
+      serviceMaster = new BspServiceMaster<I, V, E>(
         serverPortList, sessionMsecTimeout, context, this);
-      masterThread = new MasterThread<I, V, E, M>(serviceMaster, context);
+      masterThread = new MasterThread<I, V, E>(serviceMaster, context);
       masterThread.start();
     }
     if (graphFunctions.isWorker()) {
       if (LOG.isInfoEnabled()) {
         LOG.info("setup: Starting up BspServiceWorker...");
       }
-      serviceWorker = new BspServiceWorker<I, V, E, M>(
+      serviceWorker = new BspServiceWorker<I, V, E>(
         serverPortList,
         sessionMsecTimeout,
         context,
@@ -729,8 +719,8 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
    */
   private void processGraphPartitions(final Mapper<?, ?, ?, ?>.Context context,
       List<PartitionStats> partitionStatsList,
-      final GraphState<I, V, E, M> graphState,
-      final MessageStoreByPartition<I, M> messageStore,
+      final GraphState graphState,
+      final MessageStoreByPartition<I, Writable> messageStore,
       int numPartitions,
       int numThreads) {
     final BlockingQueue<Integer> computePartitionIdQueue =
@@ -748,7 +738,7 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
           @Override
           public Callable<Collection<PartitionStats>> newCallable(
               int callableId) {
-            return new ComputeCallable<I, V, E, M>(
+            return new ComputeCallable<I, V, E, Writable, Writable>(
                 context,
                 graphState,
                 messageStore,
@@ -769,14 +759,12 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
 
   /**
    * Handle the event that this superstep is a restart of a failed one.
-   * @param aggregatorUsage aggregator metadata
    * @param superstep current superstep
    * @param graphState the BSP graph state
    * @return the graph state, updated if this is a restart superstep
    */
-  private GraphState<I, V, E, M> checkSuperstepRestarted(
-    WorkerAggregatorUsage aggregatorUsage, long superstep,
-    GraphState<I, V, E, M> graphState) throws IOException {
+  private GraphState checkSuperstepRestarted(long superstep,
+    GraphState graphState) throws IOException {
     // Might need to restart from another superstep
     // (manually or automatic), or store a checkpoint
     if (serviceWorker.getRestartedSuperstep() == superstep) {
@@ -788,10 +776,10 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
       finishedSuperstepStats = new FinishedSuperstepStats(0, false,
           vertexEdgeCount.getVertexCount(), vertexEdgeCount.getEdgeCount(),
           false);
-      graphState = new GraphState<I, V, E, M>(superstep,
+      graphState = new GraphState(superstep,
           finishedSuperstepStats.getVertexCount(),
           finishedSuperstepStats.getEdgeCount(),
-          context, this, null, aggregatorUsage);
+          context);
     } else if (serviceWorker.checkpointFrequencyMet(superstep)) {
       serviceWorker.storeCheckpoint();
     }
@@ -927,4 +915,8 @@ public class GraphTaskManager<I extends WritableComparable, V extends Writable,
           "original expection will be rethrown", e1);
     }
   }
+
+  public ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
+    return conf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java b/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
index 61624e5..82fbe0c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
@@ -28,30 +28,21 @@ import org.apache.giraph.edge.MutableEdgesWrapper;
 import org.apache.giraph.edge.MutableOutEdges;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.edge.StrictRandomAccessOutEdges;
-import org.apache.giraph.partition.PartitionContext;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
-import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.mapreduce.Mapper;
 
-import java.io.IOException;
 import java.util.Iterator;
 
 /**
- * Basic abstract class for writing a BSP application for computation.
- * Giraph will store Vertex value and edges, hence all user data should
- * be stored as part of the vertex value.
+ * Class which holds vertex id, data and edges.
  *
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
-public abstract class Vertex<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, M>
-    implements WorkerAggregatorUsage {
+public class Vertex<I extends WritableComparable,
+    V extends Writable, E extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E> {
   /** Vertex id. */
   private I id;
   /** Vertex value. */
@@ -60,8 +51,6 @@ public abstract class Vertex<I extends WritableComparable,
   private OutEdges<I, E> edges;
   /** If true, do not do anymore computation on this vertex. */
   private boolean halt;
-  /** Global graph state **/
-  private GraphState<I, V, E, M> graphState;
 
   /**
    * Initialize id, value, and edges.
@@ -109,25 +98,6 @@ public abstract class Vertex<I extends WritableComparable,
   }
 
   /**
-   * Must be defined by user to do computation on a single Vertex.
-   *
-   * @param messages Messages that were sent to this vertex in the previous
-   *                 superstep.  Each message is only guaranteed to have
-   *                 a life expectancy as long as next() is not called.
-   * @throws IOException
-   */
-  public abstract void compute(Iterable<M> messages) throws IOException;
-
-  /**
-   * Retrieves the current superstep.
-   *
-   * @return Current superstep
-   */
-  public long getSuperstep() {
-    return graphState.getSuperstep();
-  }
-
-  /**
    * Get the vertex id.
    *
    * @return My vertex id.
@@ -155,26 +125,6 @@ public abstract class Vertex<I extends WritableComparable,
   }
 
   /**
-   * Get the total (all workers) number of vertices that
-   * existed in the previous superstep.
-   *
-   * @return Total number of vertices (-1 if first superstep)
-   */
-  public long getTotalNumVertices() {
-    return graphState.getTotalNumVertices();
-  }
-
-  /**
-   * Get the total (all workers) number of edges that
-   * existed in the previous superstep.
-   *
-   * @return Total number of edges (-1 if first superstep)
-   */
-  public long getTotalNumEdges() {
-    return graphState.getTotalNumEdges();
-  }
-
-  /**
    * Get a read-only view of the out-edges of this vertex.
    * Note: edge objects returned by this iterable may be invalidated as soon
    * as the next element is requested. Thus, keeping a reference to an edge
@@ -329,32 +279,6 @@ public abstract class Vertex<I extends WritableComparable,
   }
 
   /**
-   * Send a message to a vertex id.  The message should not be mutated after
-   * this method returns or else undefined results could occur.
-   *
-   * @param id Vertex id to send the message to
-   * @param message Message data to send.  Note that after the message is sent,
-   *        the user should not modify the object.
-   */
-  public void sendMessage(I id, M message) {
-    if (graphState.getWorkerClientRequestProcessor().
-          sendMessageRequest(id, message)) {
-      graphState.getGraphTaskManager().notifySentMessages();
-    }
-  }
-
-  /**
-   * Send a message to all edges.
-   *
-   * @param message Message sent to all edges.
-   */
-  public void sendMessageToAllEdges(M message) {
-    for (Edge<I, E> edge : getEdges()) {
-      sendMessage(edge.getTargetVertexId(), message);
-    }
-  }
-
-  /**
    * After this is called, the compute() code will no longer be called for
    * this vertex unless a message is sent to it.  Then the compute() code
    * will be called once again until this function is called.  The
@@ -398,115 +322,6 @@ public abstract class Vertex<I extends WritableComparable,
     edges.remove(targetVertexId);
   }
 
-  /**
-   * Sends a request to create a vertex that will be available during the
-   * next superstep.
-   *
-   * @param id Vertex id
-   * @param value Vertex value
-   * @param edges Initial edges
-   */
-  public void addVertexRequest(I id, V value, OutEdges<I, E> edges)
-    throws IOException {
-    Vertex<I, V, E, M> vertex = getConf().createVertex();
-    vertex.initialize(id, value, edges);
-    graphState.getWorkerClientRequestProcessor().addVertexRequest(vertex);
-  }
-
-  /**
-   * Sends a request to create a vertex that will be available during the
-   * next superstep.
-   *
-   * @param id Vertex id
-   * @param value Vertex value
-   */
-  public void addVertexRequest(I id, V value) throws IOException {
-    addVertexRequest(id, value, getConf().createAndInitializeOutEdges());
-  }
-
-  /**
-   * Request to remove a vertex from the graph
-   * (applied just prior to the next superstep).
-   *
-   * @param vertexId Id of the vertex to be removed.
-   */
-  public void removeVertexRequest(I vertexId) throws IOException {
-    graphState.getWorkerClientRequestProcessor().
-        removeVertexRequest(vertexId);
-  }
-
-  /**
-   * Request to add an edge of a vertex in the graph
-   * (processed just prior to the next superstep)
-   *
-   * @param sourceVertexId Source vertex id of edge
-   * @param edge Edge to add
-   */
-  public void addEdgeRequest(I sourceVertexId, Edge<I, E> edge)
-    throws IOException {
-    graphState.getWorkerClientRequestProcessor().
-        addEdgeRequest(sourceVertexId, edge);
-  }
-
-  /**
-   * Request to remove all edges from a given source vertex to a given target
-   * vertex (processed just prior to the next superstep).
-   *
-   * @param sourceVertexId Source vertex id
-   * @param targetVertexId Target vertex id
-   */
-  public void removeEdgesRequest(I sourceVertexId, I targetVertexId)
-    throws IOException {
-    graphState.getWorkerClientRequestProcessor().
-        removeEdgesRequest(sourceVertexId, targetVertexId);
-  }
-
-  /**
-   * Set the graph state for all workers
-   *
-   * @param graphState Graph state for all workers
-   */
-  public void setGraphState(GraphState<I, V, E, M> graphState) {
-    this.graphState = graphState;
-  }
-
-  /**
-   * Get the mapper context
-   *
-   * @return Mapper context
-   */
-  public Mapper.Context getContext() {
-    return graphState.getContext();
-  }
-
-  /**
-   * Get the partition context
-   *
-   * @return Partition context
-   */
-  public PartitionContext getPartitionContext() {
-    return graphState.getPartitionContext();
-  }
-
-  /**
-   * Get the worker context
-   *
-   * @return WorkerContext context
-   */
-  public WorkerContext getWorkerContext() {
-    return graphState.getGraphTaskManager().getWorkerContext();
-  }
-
-  @Override
-  public <A extends Writable> void aggregate(String name, A value) {
-    graphState.getWorkerAggregatorUsage().aggregate(name, value);
-  }
-
-  @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    return graphState.getWorkerAggregatorUsage().<A>getAggregatedValue(name);
-  }
-
   @Override
   public String toString() {
     return "Vertex(id=" + getId() + ",value=" + getValue() +

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/VertexChanges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/VertexChanges.java b/giraph-core/src/main/java/org/apache/giraph/graph/VertexChanges.java
index 9474636..3d09c06 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/VertexChanges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/VertexChanges.java
@@ -31,18 +31,17 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public interface VertexChanges<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /**
    * Get the added vertices for this particular vertex index from the previous
    * superstep.
    *
    * @return List of vertices for this vertex index.
    */
-  List<Vertex<I, V, E, M>> getAddedVertexList();
+  List<Vertex<I, V, E>> getAddedVertexList();
 
   /**
    * Get the number of times this vertex was removed in the previous

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java b/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
index 75c0aef..6f54dc7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
@@ -41,15 +41,13 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public class VertexMutations<I extends WritableComparable,
-    V extends Writable, E extends Writable,
-    M extends Writable> implements VertexChanges<I, V, E, M>,
+    V extends Writable, E extends Writable> implements VertexChanges<I, V, E>,
     Writable, ImmutableClassesGiraphConfigurable {
   /** List of added vertices during the last superstep */
-  private final List<Vertex<I, V, E, M>> addedVertexList = Lists.newArrayList();
+  private final List<Vertex<I, V, E>> addedVertexList = Lists.newArrayList();
   /** Count of remove vertex requests */
   private int removedVertexCount = 0;
   /** List of added edges */
@@ -57,15 +55,15 @@ public class VertexMutations<I extends WritableComparable,
   /** List of removed edges */
   private final List<I> removedEdgeList = Lists.newArrayList();
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
 
   /**
    * Copy the vertex mutations.
    *
    * @return Copied vertex mutations
    */
-  public VertexMutations<I, V, E, M> copy() {
-    VertexMutations<I, V, E, M> copied = new VertexMutations<I, V, E, M>();
+  public VertexMutations<I, V, E> copy() {
+    VertexMutations<I, V, E> copied = new VertexMutations<I, V, E>();
     copied.addedVertexList.addAll(this.addedVertexList);
     copied.removedVertexCount = this.removedVertexCount;
     copied.addedEdgeList.addAll(this.addedEdgeList);
@@ -75,7 +73,7 @@ public class VertexMutations<I extends WritableComparable,
   }
 
   @Override
-  public List<Vertex<I, V, E, M>> getAddedVertexList() {
+  public List<Vertex<I, V, E>> getAddedVertexList() {
     return addedVertexList;
   }
 
@@ -87,7 +85,7 @@ public class VertexMutations<I extends WritableComparable,
 
     int addedVertexListSize = input.readInt();
     for (int i = 0; i < addedVertexListSize; ++i) {
-      Vertex<I, V, E, M> vertex =
+      Vertex<I, V, E> vertex =
           WritableUtils.readVertexFromDataInput(input, getConf());
       addedVertexList.add(vertex);
     }
@@ -109,7 +107,7 @@ public class VertexMutations<I extends WritableComparable,
   @Override
   public void write(DataOutput output) throws IOException {
     output.writeInt(addedVertexList.size());
-    for (Vertex<I, V, E, M> vertex : addedVertexList) {
+    for (Vertex<I, V, E> vertex : addedVertexList) {
       WritableUtils.writeVertexToDataOutput(output, vertex, getConf());
     }
     output.writeInt(removedVertexCount);
@@ -129,7 +127,7 @@ public class VertexMutations<I extends WritableComparable,
    *
    * @param vertex Vertex to be added
    */
-  public void addVertex(Vertex<I, V, E, M> vertex) {
+  public void addVertex(Vertex<I, V, E> vertex) {
     addedVertexList.add(vertex);
   }
 
@@ -178,7 +176,7 @@ public class VertexMutations<I extends WritableComparable,
    *
    * @param vertexMutations Object to be added
    */
-  public void addVertexMutations(VertexMutations<I, V, E, M> vertexMutations) {
+  public void addVertexMutations(VertexMutations<I, V, E> vertexMutations) {
     addedVertexList.addAll(vertexMutations.getAddedVertexList());
     removedVertexCount += vertexMutations.getRemovedVertexCount();
     addedEdgeList.addAll(vertexMutations.getAddedEdgeList());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/VertexResolver.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/VertexResolver.java b/giraph-core/src/main/java/org/apache/giraph/graph/VertexResolver.java
index 1fc0ddc..b6659f4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/VertexResolver.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/VertexResolver.java
@@ -28,12 +28,10 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface VertexResolver<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends GraphStateAware<I, V, E, M> {
+    V extends Writable, E extends Writable> {
   /**
    * A vertex may have been removed, created zero or more times and had
    * zero or more messages sent to it.  This method will handle all situations
@@ -47,15 +45,8 @@ public interface VertexResolver<I extends WritableComparable,
    * @return Vertex to be returned, if null, and a vertex currently exists
    *         it will be removed
    */
-  Vertex<I, V, E, M> resolve(I vertexId,
-      Vertex<I, V, E, M> vertex,
-      VertexChanges<I, V, E, M> vertexChanges,
+  Vertex<I, V, E> resolve(I vertexId,
+      Vertex<I, V, E> vertex,
+      VertexChanges<I, V, E> vertexChanges,
       boolean hasMessages);
-
-  /**
-   * Set the graph state.
-   *
-   * @param graphState Graph state saved.
-   */
-  void setGraphState(GraphState<I, V, E, M> graphState);
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java b/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java
index e62bb01..eb9197c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java
@@ -36,7 +36,7 @@ public interface VertexValueFactory<V extends Writable> {
    * @param configuration Configuration
    */
   void initialize(
-      ImmutableClassesGiraphConfiguration<?, V, ?, ?> configuration);
+      ImmutableClassesGiraphConfiguration<?, V, ?> configuration);
 
   /**
    * Create a new vertex value.

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java b/giraph-core/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java
index c03d718..ebc62f6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/integration/SuperstepHashPartitionerFactory.java
@@ -42,12 +42,11 @@ import org.apache.log4j.Logger;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class SuperstepHashPartitionerFactory<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends HashPartitionerFactory<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends HashPartitionerFactory<I, V, E> {
   /**
    * Changes the {@link HashMasterPartitioner} to make ownership of the
    * partitions based on a superstep.  For testing only as it is totally
@@ -56,11 +55,10 @@ public class SuperstepHashPartitionerFactory<I extends WritableComparable,
    * @param <I> vertex id
    * @param <V> vertex data
    * @param <E> edge data
-   * @param <M> message data
    */
   private static class SuperstepMasterPartition<I extends WritableComparable,
-      V extends Writable, E extends Writable, M extends Writable>
-      extends HashMasterPartitioner<I, V, E, M> {
+      V extends Writable, E extends Writable>
+      extends HashMasterPartitioner<I, V, E> {
     /** Class logger */
     private static Logger LOG =
         Logger.getLogger(SuperstepMasterPartition.class);
@@ -120,8 +118,8 @@ public class SuperstepHashPartitionerFactory<I extends WritableComparable,
   }
 
   @Override
-  public MasterGraphPartitioner<I, V, E, M>
+  public MasterGraphPartitioner<I, V, E>
   createMasterGraphPartitioner() {
-    return new SuperstepMasterPartition<I, V, E, M>(getConf());
+    return new SuperstepMasterPartition<I, V, E>(getConf());
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java b/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
index 6d87bf2..a1af98f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/EdgeReader.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 @SuppressWarnings("rawtypes")
 public abstract class EdgeReader<I extends WritableComparable,
     E extends Writable> extends DefaultImmutableClassesGiraphConfigurable<
-        I, Writable, E, Writable> {
+        I, Writable, E> {
   /**
    * Use the input split and context to setup reading the edges.
    * Guaranteed to be called prior to any other function.

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/GiraphInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/GiraphInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/GiraphInputFormat.java
index 58d79a6..86e86d8 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/GiraphInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/GiraphInputFormat.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.util.ReflectionUtils;
  */
 public abstract class GiraphInputFormat<I extends WritableComparable,
     V extends Writable, E extends Writable> extends
-    DefaultImmutableClassesGiraphConfigurable<I, V, E, Writable> {
+    DefaultImmutableClassesGiraphConfigurable<I, V, E> {
   /**
    * Get the list of input splits for the format.
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/SimpleVertexWriter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/SimpleVertexWriter.java b/giraph-core/src/main/java/org/apache/giraph/io/SimpleVertexWriter.java
index e4c3496..41a049f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/SimpleVertexWriter.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/SimpleVertexWriter.java
@@ -40,6 +40,6 @@ public interface SimpleVertexWriter<I extends WritableComparable,
    * @throws IOException
    * @throws InterruptedException
    */
-  void writeVertex(Vertex<I, V, E, ?> vertex) throws IOException,
+  void writeVertex(Vertex<I, V, E> vertex) throws IOException,
       InterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/VertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexOutputFormat.java
index 154f7e4..ad00a8e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexOutputFormat.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 public abstract class VertexOutputFormat<
     I extends WritableComparable, V extends Writable,
     E extends Writable> extends
-    DefaultImmutableClassesGiraphConfigurable<I, V, E, Writable> {
+    DefaultImmutableClassesGiraphConfigurable<I, V, E> {
   /**
    * Create a vertex writer for a given split. The framework will call
    * {@link VertexWriter#initialize(TaskAttemptContext)} before

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
index 3f6bb3f..9695169 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 @SuppressWarnings("rawtypes")
 public abstract class VertexReader<I extends WritableComparable,
     V extends Writable, E extends Writable> extends
-    DefaultImmutableClassesGiraphConfigurable<I, V, E, Writable> {
+    DefaultImmutableClassesGiraphConfigurable<I, V, E> {
   /**
    * Use the input split and context to setup reading the vertices.
    * Guaranteed to be called prior to any other function.
@@ -68,7 +68,7 @@ public abstract class VertexReader<I extends WritableComparable,
    * @throws IOException
    * @throws InterruptedException
    */
-  public abstract Vertex<I, V, E, ?> getCurrentVertex()
+  public abstract Vertex<I, V, E> getCurrentVertex()
     throws IOException, InterruptedException;
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
index 0b06a4a..70e721e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
@@ -39,9 +39,9 @@ public abstract class VertexValueReader<I extends WritableComparable,
   }
 
   @Override
-  public final Vertex<I, V, Writable, ?> getCurrentVertex() throws IOException,
+  public final Vertex<I, V, Writable> getCurrentVertex() throws IOException,
       InterruptedException {
-    Vertex<I, V, Writable, ?> vertex = getConf().createVertex();
+    Vertex<I, V, Writable> vertex = getConf().createVertex();
     vertex.initialize(getCurrentVertexId(), getCurrentVertexValue());
     return vertex;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/VertexWriter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexWriter.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexWriter.java
index a4285c1..69fdfc5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexWriter.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexWriter.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 @SuppressWarnings("rawtypes")
 public abstract class VertexWriter<I extends WritableComparable,
     V extends Writable, E extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
     implements SimpleVertexWriter<I, V, E> {
   /**
    * Use the context to setup writing the vertices.

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultEdgeInputFilter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultEdgeInputFilter.java b/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultEdgeInputFilter.java
index ad52496..40ab922 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultEdgeInputFilter.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultEdgeInputFilter.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public class DefaultEdgeInputFilter<I extends WritableComparable,
     E extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E, Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E>
     implements EdgeInputFilter<I, E> {
   @Override
   public boolean dropEdge(I sourceId, Edge<I, E> edge) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultVertexInputFilter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultVertexInputFilter.java b/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultVertexInputFilter.java
index 2976cbc..f35ade5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultVertexInputFilter.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/filters/DefaultVertexInputFilter.java
@@ -32,10 +32,10 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public class DefaultVertexInputFilter<I extends WritableComparable,
     V extends Writable, E extends Writable, M extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, M>
-    implements VertexInputFilter<I, V, E, M> {
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
+    implements VertexInputFilter<I, V, E> {
   @Override
-  public boolean dropVertex(Vertex<I, V, E, M> vertex) {
+  public boolean dropVertex(Vertex<I, V, E> vertex) {
     return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/filters/VertexInputFilter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/filters/VertexInputFilter.java b/giraph-core/src/main/java/org/apache/giraph/io/filters/VertexInputFilter.java
index d9af103..c7d178e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/filters/VertexInputFilter.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/filters/VertexInputFilter.java
@@ -27,16 +27,15 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex ID
  * @param <V> Vertex Value
  * @param <E> Edge Value
- * @param <M> Message Value
  */
 public interface VertexInputFilter<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /**
    * Whether to drop a vertex on input.
    *
    * @param vertex to check
    * @return true if we should drop vertex
    */
-  boolean dropVertex(Vertex<I, V, E, M> vertex);
+  boolean dropVertex(Vertex<I, V, E> vertex);
 }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexOutputFormat.java
index f71ef25..8efbfd0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexOutputFormat.java
@@ -69,7 +69,7 @@ public class AdjacencyListTextVertexOutputFormat<I extends WritableComparable,
     }
 
     @Override
-    public Text convertVertexToLine(Vertex<I, V, E, ?> vertex)
+    public Text convertVertexToLine(Vertex<I, V, E> vertex)
       throws IOException {
       StringBuffer sb = new StringBuffer(vertex.getId().toString());
       sb.append(delimiter);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/IdWithValueTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/IdWithValueTextOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/IdWithValueTextOutputFormat.java
index 6dd7468..bd69586 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/IdWithValueTextOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/IdWithValueTextOutputFormat.java
@@ -75,7 +75,7 @@ public class IdWithValueTextOutputFormat<I extends WritableComparable,
     }
 
     @Override
-    protected Text convertVertexToLine(Vertex<I, V, E, ?> vertex)
+    protected Text convertVertexToLine(Vertex<I, V, E> vertex)
       throws IOException {
       String first;
       String second;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntTextVertexValueInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntTextVertexValueInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntTextVertexValueInputFormat.java
index a7dbef8..1038a32 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntTextVertexValueInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntTextVertexValueInputFormat.java
@@ -35,11 +35,9 @@ import java.util.regex.Pattern;
  * Each line consists of: id, value
  *
  * @param <E> Edge value
- * @param <M> Message data
  */
-public class IntIntTextVertexValueInputFormat<E extends Writable,
-    M extends Writable> extends
-    TextVertexValueInputFormat<IntWritable, IntWritable, E, M> {
+public class IntIntTextVertexValueInputFormat<E extends Writable> extends
+    TextVertexValueInputFormat<IntWritable, IntWritable, E> {
   /** Separator for id and value */
   private static final Pattern SEPARATOR = Pattern.compile("[\t ]");
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexOutputFormat.java
index 7d8fcf6..a71cf96 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexOutputFormat.java
@@ -60,7 +60,7 @@ public class JsonBase64VertexOutputFormat<I extends WritableComparable,
   protected class JsonBase64VertexWriter extends TextVertexWriterToEachLine {
 
     @Override
-    protected Text convertVertexToLine(Vertex<I, V, E, ?> vertex)
+    protected Text convertVertexToLine(Vertex<I, V, E> vertex)
       throws IOException {
       ByteArrayOutputStream outputStream =
           new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
index 7dfd607..112860c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
@@ -95,9 +95,8 @@ public class JsonLongDoubleFloatDoubleVertexInputFormat extends
     }
 
     @Override
-    protected Vertex<LongWritable, DoubleWritable, FloatWritable,
-              DoubleWritable> handleException(Text line, JSONArray jsonVertex,
-                  JSONException e) {
+    protected Vertex<LongWritable, DoubleWritable, FloatWritable>
+    handleException(Text line, JSONArray jsonVertex, JSONException e) {
       throw new IllegalArgumentException(
           "Couldn't get vertex from line " + line, e);
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexOutputFormat.java
index d0a3305..13c80aa 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexOutputFormat.java
@@ -52,8 +52,7 @@ public class JsonLongDoubleFloatDoubleVertexOutputFormat extends
     TextVertexWriterToEachLine {
     @Override
     public Text convertVertexToLine(
-      Vertex<LongWritable, DoubleWritable,
-        FloatWritable, ?> vertex
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex
     ) throws IOException {
       JSONArray jsonVertex = new JSONArray();
       try {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
index c9390ba..d8abfdb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
@@ -116,9 +116,9 @@ public class PseudoRandomIntNullVertexInputFormat extends
     }
 
     @Override
-    public Vertex<IntWritable, FloatWritable, NullWritable, ?>
+    public Vertex<IntWritable, FloatWritable, NullWritable>
     getCurrentVertex() throws IOException, InterruptedException {
-      Vertex<IntWritable, FloatWritable, NullWritable, ?> vertex =
+      Vertex<IntWritable, FloatWritable, NullWritable> vertex =
           getConf().createVertex();
       int vertexId = startingVertexId + verticesRead;
       OutEdges<IntWritable, NullWritable> edges =

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
index 5d293eb..91a19e6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomVertexInputFormat.java
@@ -133,9 +133,9 @@ public class PseudoRandomVertexInputFormat extends
     }
 
     @Override
-    public Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+    public Vertex<LongWritable, DoubleWritable, DoubleWritable>
     getCurrentVertex() throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+      Vertex<LongWritable, DoubleWritable, DoubleWritable>
       vertex = getConf().createVertex();
       long vertexId = startingVertexId + verticesRead;
       // Seed on the vertex id to keep the vertex data the same when

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
index 0f2d929..1d31f4f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
  */
 @SuppressWarnings("rawtypes")
 public class SequenceFileVertexInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, X extends Vertex<I, V, E, ?>>
+    V extends Writable, E extends Writable, X extends Vertex<I, V, E>>
     extends VertexInputFormat<I, V, E> {
   /** Internal input format */
   protected SequenceFileInputFormat<I, X> sequenceFileInputFormat =
@@ -68,7 +68,7 @@ public class SequenceFileVertexInputFormat<I extends WritableComparable,
    * @param <X> Value type
    */
   public static class SequenceFileVertexReader<I extends WritableComparable,
-      V extends Writable, E extends Writable, X extends Vertex<I, V, E, ?>>
+      V extends Writable, E extends Writable, X extends Vertex<I, V, E>>
       extends VertexReader<I, V, E> {
     /** Internal record reader from {@link SequenceFileInputFormat} */
     private final RecordReader<I, X> recordReader;
@@ -92,7 +92,7 @@ public class SequenceFileVertexInputFormat<I extends WritableComparable,
       return recordReader.nextKeyValue();
     }
 
-    @Override public Vertex<I, V, E, ?> getCurrentVertex()
+    @Override public Vertex<I, V, E> getCurrentVertex()
       throws IOException, InterruptedException {
       return recordReader.getCurrentValue();
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexOutputFormat.java
index c4ed65c..c6ff6d4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexOutputFormat.java
@@ -105,7 +105,7 @@ public abstract class SequenceFileVertexOutputFormat<
     }
 
     @Override
-    public final void writeVertex(Vertex<I, V, E, ?> vertex) throws
+    public final void writeVertex(Vertex<I, V, E> vertex) throws
       IOException, InterruptedException {
       // Convert vertex id to type OK.
       OK outKey = convertToSequenceFileKey(vertex.getId());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/TextDoubleDoubleAdjacencyListVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextDoubleDoubleAdjacencyListVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextDoubleDoubleAdjacencyListVertexInputFormat.java
index 6e62b71..17174a3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextDoubleDoubleAdjacencyListVertexInputFormat.java
@@ -21,7 +21,6 @@ import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
@@ -29,10 +28,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * Class to read graphs stored as adjacency lists with ids represented by
  * Strings and values as doubles.  This is a good inputformat for reading
  * graphs where the id types do not matter and can be stashed in a String.
- *
- * @param <M> Message type.
  */
-public class TextDoubleDoubleAdjacencyListVertexInputFormat<M extends Writable>
+public class TextDoubleDoubleAdjacencyListVertexInputFormat
     extends AdjacencyListTextVertexInputFormat<Text, DoubleWritable,
             DoubleWritable>  {
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
index b08e6f7..debdccc 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
@@ -154,10 +154,10 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
     TextVertexReader {
 
     @Override
-    public final Vertex<I, V, E, ?> getCurrentVertex() throws IOException,
+    public final Vertex<I, V, E> getCurrentVertex() throws IOException,
     InterruptedException {
       Text line = getRecordReader().getCurrentValue();
-      Vertex<I, V, E, ?> vertex = getConf().createVertex();
+      Vertex<I, V, E> vertex = getConf().createVertex();
       vertex.initialize(getId(line), getValue(line), getEdges(line));
       return vertex;
     }
@@ -222,10 +222,10 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
     }
 
     @Override
-    public final Vertex<I, V, E, ?> getCurrentVertex() throws IOException,
+    public final Vertex<I, V, E> getCurrentVertex() throws IOException,
     InterruptedException {
       Text line = getRecordReader().getCurrentValue();
-      Vertex<I, V, E, ?> vertex;
+      Vertex<I, V, E> vertex;
       T processed = preprocessLine(line);
       vertex = getConf().createVertex();
       vertex.initialize(getId(processed), getValue(processed),
@@ -306,11 +306,11 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
 
     @SuppressWarnings("unchecked")
     @Override
-    public final Vertex<I, V, E, ?> getCurrentVertex() throws IOException,
+    public final Vertex<I, V, E> getCurrentVertex() throws IOException,
         InterruptedException {
       // Note we are reading from value only since key is the line number
       Text line = getRecordReader().getCurrentValue();
-      Vertex<I, V, E, ?> vertex;
+      Vertex<I, V, E> vertex;
       T processed = null;
       try {
         processed = preprocessLine(line);
@@ -401,7 +401,7 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
      *          the exception thrown while reading the line
      * @return the recovered/alternative vertex to be used
      */
-    protected Vertex<I, V, E, ?> handleException(Text line, T processed, X e) {
+    protected Vertex<I, V, E> handleException(Text line, T processed, X e) {
       throw new IllegalArgumentException(e);
     }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexOutputFormat.java
index a3073f9..c91d543 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexOutputFormat.java
@@ -158,7 +158,7 @@ public abstract class TextVertexOutputFormat<I extends WritableComparable,
      * @throws IOException
      *           exception that can be thrown while writing
      */
-    protected abstract Text convertVertexToLine(Vertex<I, V, E, ?> vertex)
+    protected abstract Text convertVertexToLine(Vertex<I, V, E> vertex)
       throws IOException;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexValueInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexValueInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexValueInputFormat.java
index 6d133ae..e960444 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexValueInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexValueInputFormat.java
@@ -39,11 +39,10 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public abstract class TextVertexValueInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     extends VertexValueInputFormat<I, V> {
   /** Uses the GiraphTextInputFormat to do everything */
   protected GiraphTextInputFormat textInputFormat = new GiraphTextInputFormat();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/EdgeInputFormatDescription.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/EdgeInputFormatDescription.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/EdgeInputFormatDescription.java
index 569cee9..4b48e63 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/EdgeInputFormatDescription.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/EdgeInputFormatDescription.java
@@ -78,11 +78,11 @@ public class EdgeInputFormatDescription<I extends WritableComparable,
    * @param conf Configuration which we want to create a copy from
    * @return Copy of configuration
    */
-  private ImmutableClassesGiraphConfiguration<I, Writable, E, Writable>
+  private ImmutableClassesGiraphConfiguration<I, Writable, E>
   createConfigurationCopy(
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> conf) {
-    ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> confCopy =
-        new ImmutableClassesGiraphConfiguration<I, Writable, E, Writable>(conf);
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
+    ImmutableClassesGiraphConfiguration<I, Writable, E> confCopy =
+        new ImmutableClassesGiraphConfiguration<I, Writable, E>(conf);
     confCopy.setEdgeInputFormatClass(getInputFormatClass());
     putParametersToConfiguration(confCopy);
     return confCopy;
@@ -130,13 +130,13 @@ public class EdgeInputFormatDescription<I extends WritableComparable,
    */
   public static <I extends WritableComparable,
       E extends Writable> List<EdgeInputFormat<I, E>> createEdgeInputFormats(
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
     List<EdgeInputFormatDescription<I, E>> descriptions =
         getEdgeInputFormatDescriptions(conf);
     List<EdgeInputFormat<I, E>> edgeInputFormats =
         Lists.newArrayListWithCapacity(descriptions.size());
     for (EdgeInputFormatDescription<I, E> description : descriptions) {
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> confCopy =
+      ImmutableClassesGiraphConfiguration<I, Writable, E> confCopy =
           description.createConfigurationCopy(conf);
       edgeInputFormats.add(confCopy.createWrappedEdgeInputFormat());
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiEdgeInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiEdgeInputFormat.java
index c377fbc..fa8839b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiEdgeInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiEdgeInputFormat.java
@@ -48,7 +48,7 @@ public class MultiEdgeInputFormat<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
     super.setConf(conf);
     edgeInputFormats =
         EdgeInputFormatDescription.createEdgeInputFormats(getConf());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiVertexInputFormat.java
index 72929d9..e851e38 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/MultiVertexInputFormat.java
@@ -49,7 +49,7 @@ public class MultiVertexInputFormat<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     super.setConf(conf);
     vertexInputFormats =
         VertexInputFormatDescription.createVertexInputFormats(getConf());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/VertexInputFormatDescription.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/VertexInputFormatDescription.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/VertexInputFormatDescription.java
index bdd5a74..1487749 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/VertexInputFormatDescription.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/multi/VertexInputFormatDescription.java
@@ -80,11 +80,11 @@ public class VertexInputFormatDescription<I extends WritableComparable,
    * @param conf Configuration which we want to create a copy from
    * @return Copy of configuration
    */
-  private ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+  private ImmutableClassesGiraphConfiguration<I, V, E>
   createConfigurationCopy(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
-    ImmutableClassesGiraphConfiguration<I, V, E, Writable> confCopy =
-        new ImmutableClassesGiraphConfiguration<I, V, E, Writable>(conf);
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
+    ImmutableClassesGiraphConfiguration<I, V, E> confCopy =
+        new ImmutableClassesGiraphConfiguration<I, V, E>(conf);
     confCopy.setVertexInputFormatClass(getInputFormatClass());
     putParametersToConfiguration(confCopy);
     return confCopy;
@@ -136,13 +136,13 @@ public class VertexInputFormatDescription<I extends WritableComparable,
   public static <I extends WritableComparable, V extends Writable,
       E extends Writable>
   List<VertexInputFormat<I, V, E>> createVertexInputFormats(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     List<VertexInputFormatDescription<I, V, E>> descriptions =
         getVertexInputFormatDescriptions(conf);
     List<VertexInputFormat<I, V, E>> vertexInputFormats =
         Lists.newArrayListWithCapacity(descriptions.size());
     for (VertexInputFormatDescription<I, V, E> description : descriptions) {
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> confCopy =
+      ImmutableClassesGiraphConfiguration<I, V, E> confCopy =
           description.createConfigurationCopy(conf);
       vertexInputFormats.add(confCopy.createWrappedVertexInputFormat());
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
index c0a2cd1..aae7a72 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedEdgeReader.java
@@ -50,7 +50,7 @@ public class WrappedEdgeReader<I extends WritableComparable,
    * @param conf Configuration
    */
   public WrappedEdgeReader(EdgeReader<I, E> baseEdgeReader,
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
     this.baseEdgeReader = baseEdgeReader;
     super.setConf(conf);
     baseEdgeReader.setConf(conf);
@@ -58,7 +58,7 @@ public class WrappedEdgeReader<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
     // We don't want to use external configuration
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexOutputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexOutputFormat.java
index 8110209..bffa330 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexOutputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexOutputFormat.java
@@ -70,7 +70,7 @@ public class WrappedVertexOutputFormat<I extends WritableComparable,
     return new VertexWriter<I, V, E>() {
       @Override
       public void setConf(
-          ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+          ImmutableClassesGiraphConfiguration<I, V, E> conf) {
         super.setConf(conf);
         vertexWriter.setConf(conf);
       }
@@ -91,7 +91,7 @@ public class WrappedVertexOutputFormat<I extends WritableComparable,
 
       @Override
       public void writeVertex(
-          Vertex<I, V, E, ?> vertex) throws IOException, InterruptedException {
+          Vertex<I, V, E> vertex) throws IOException, InterruptedException {
         vertexWriter.writeVertex(vertex);
       }
     };

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
index 3a8ac50..54adfec 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/internal/WrappedVertexReader.java
@@ -51,7 +51,7 @@ public class WrappedVertexReader<I extends WritableComparable,
    * @param conf Configuration
    */
   public WrappedVertexReader(VertexReader<I, V, E> baseVertexReader,
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     this.baseVertexReader = baseVertexReader;
     super.setConf(conf);
     baseVertexReader.setConf(conf);
@@ -59,7 +59,7 @@ public class WrappedVertexReader<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     // We don't want to use external configuration
   }
 
@@ -76,7 +76,7 @@ public class WrappedVertexReader<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, ?> getCurrentVertex() throws IOException,
+  public Vertex<I, V, E> getCurrentVertex() throws IOException,
       InterruptedException {
     return baseVertexReader.getCurrentVertex();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/iterables/EdgeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/iterables/EdgeReaderWrapper.java b/giraph-core/src/main/java/org/apache/giraph/io/iterables/EdgeReaderWrapper.java
index efd8fe7..f61ac45 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/iterables/EdgeReaderWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/iterables/EdgeReaderWrapper.java
@@ -53,7 +53,7 @@ public class EdgeReaderWrapper<I extends WritableComparable,
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, Writable, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, Writable, E> conf) {
     super.setConf(conf);
     conf.configureIfPossible(edgeReader);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java b/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
index 614f945..ca35c51 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
@@ -38,23 +38,23 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 public class VertexReaderWrapper<I extends WritableComparable,
     V extends Writable, E extends Writable> extends VertexReader<I, V, E> {
   /** Wrapped vertex reader */
-  private GiraphReader<Vertex<I, V, E, ?>> vertexReader;
+  private GiraphReader<Vertex<I, V, E>> vertexReader;
   /** {@link VertexReader}-like wrapper of {@link #vertexReader} */
-  private IteratorToReaderWrapper<Vertex<I, V, E, ?>> iterator;
+  private IteratorToReaderWrapper<Vertex<I, V, E>> iterator;
 
   /**
    * Constructor
    *
    * @param vertexReader GiraphReader for vertices to wrap
    */
-  public VertexReaderWrapper(GiraphReader<Vertex<I, V, E, ?>> vertexReader) {
+  public VertexReaderWrapper(GiraphReader<Vertex<I, V, E>> vertexReader) {
     this.vertexReader = vertexReader;
-    iterator = new IteratorToReaderWrapper<Vertex<I, V, E, ?>>(vertexReader);
+    iterator = new IteratorToReaderWrapper<Vertex<I, V, E>>(vertexReader);
   }
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     super.setConf(conf);
     conf.configureIfPossible(vertexReader);
   }
@@ -65,7 +65,7 @@ public class VertexReaderWrapper<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, ?> getCurrentVertex() throws IOException,
+  public Vertex<I, V, E> getCurrentVertex() throws IOException,
       InterruptedException {
     return iterator.getCurrentObject();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/MultiThreadedSuperstepOutput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/MultiThreadedSuperstepOutput.java b/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/MultiThreadedSuperstepOutput.java
index a8dff87..452d93a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/MultiThreadedSuperstepOutput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/MultiThreadedSuperstepOutput.java
@@ -47,7 +47,7 @@ public class MultiThreadedSuperstepOutput<I extends WritableComparable,
   /** Mapper context */
   private final Mapper<?, ?, ?, ?>.Context context;
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, ?> configuration;
+  private ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /** Vertex output format, used to get new vertex writers */
   private final VertexOutputFormat<I, V, E> vertexOutputFormat;
   /**
@@ -65,7 +65,7 @@ public class MultiThreadedSuperstepOutput<I extends WritableComparable,
    * @param context Mapper context
    */
   public MultiThreadedSuperstepOutput(
-      ImmutableClassesGiraphConfiguration<I, V, E, ?> conf,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf,
       Mapper<?, ?, ?, ?>.Context context) {
     this.configuration = conf;
     vertexOutputFormat = conf.createWrappedVertexOutputFormat();
@@ -80,9 +80,7 @@ public class MultiThreadedSuperstepOutput<I extends WritableComparable,
     if (availableVertexWriters.isEmpty()) {
       try {
         vertexWriter = vertexOutputFormat.createVertexWriter(context);
-        vertexWriter.setConf(
-            (ImmutableClassesGiraphConfiguration<I, V, E, Writable>)
-                configuration);
+        vertexWriter.setConf(configuration);
         vertexWriter.initialize(context);
       } catch (IOException e) {
         throw new IllegalStateException("getVertexWriter: " +

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/NoOpSuperstepOutput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/NoOpSuperstepOutput.java b/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/NoOpSuperstepOutput.java
index 82684b2..be981cf 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/NoOpSuperstepOutput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/NoOpSuperstepOutput.java
@@ -40,7 +40,7 @@ public class NoOpSuperstepOutput<I extends WritableComparable,
   public SimpleVertexWriter<I, V, E> getVertexWriter() {
     return new SimpleVertexWriter<I, V, E>() {
       @Override
-      public void writeVertex(Vertex<I, V, E, ?> vertex) throws IOException,
+      public void writeVertex(Vertex<I, V, E> vertex) throws IOException,
           InterruptedException {
       }
     };

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/SynchronizedSuperstepOutput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/SynchronizedSuperstepOutput.java b/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/SynchronizedSuperstepOutput.java
index f94bd56..7f233e0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/SynchronizedSuperstepOutput.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/superstep_output/SynchronizedSuperstepOutput.java
@@ -57,14 +57,13 @@ public class SynchronizedSuperstepOutput<I extends WritableComparable,
    */
   @SuppressWarnings("unchecked")
   public SynchronizedSuperstepOutput(
-      ImmutableClassesGiraphConfiguration<I, V, E, ?> conf,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf,
       Mapper<?, ?, ?, ?>.Context context) {
     this.context = context;
     try {
       vertexWriter =
           conf.createWrappedVertexOutputFormat().createVertexWriter(context);
-      vertexWriter.setConf(
-          (ImmutableClassesGiraphConfiguration<I, V, E, Writable>) conf);
+      vertexWriter.setConf(conf);
       vertexWriter.initialize(context);
     } catch (IOException e) {
       throw new IllegalStateException("SynchronizedSuperstepOutput: " +
@@ -76,7 +75,7 @@ public class SynchronizedSuperstepOutput<I extends WritableComparable,
     simpleVertexWriter = new SimpleVertexWriter<I, V, E>() {
       @Override
       public synchronized void writeVertex(
-          Vertex<I, V, E, ?> vertex) throws IOException, InterruptedException {
+          Vertex<I, V, E> vertex) throws IOException, InterruptedException {
         vertexWriter.writeVertex(vertex);
       }
     };

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
index cc6b126..de17157 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
@@ -19,12 +19,12 @@
 package org.apache.giraph.job;
 
 import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
 import org.apache.giraph.graph.DefaultVertexValueFactory;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
@@ -51,10 +51,12 @@ import static org.apache.giraph.conf.GiraphConstants.VERTEX_RESOLVER_CLASS;
  * @param <I> the Vertex ID type
  * @param <V> the Vertex Value type
  * @param <E> the Edge Value type
- * @param <M> the Message type
+ * @param <M1> the incoming Message type
+ * @param <M2> the outgoing Message type
  */
 public class GiraphConfigurationValidator<I extends WritableComparable,
-  V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable, M1 extends Writable,
+    M2 extends Writable> {
   /**
    * Class logger object.
    */
@@ -67,8 +69,8 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
   private static final int VALUE_PARAM_INDEX = 1;
   /** E param vertex index in classList */
   private static final int EDGE_PARAM_INDEX = 2;
-  /** M param vertex index in classList */
-  private static final int MSG_PARAM_INDEX = 3;
+  /** M2 param vertex index in classList */
+  private static final int OUTGOING_MSG_PARAM_INDEX = 4;
   /** M param vertex combiner index in classList */
   private static final int MSG_COMBINER_PARAM_INDEX = 1;
   /** E param edge input format index in classList */
@@ -80,12 +82,12 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
 
   /** Vertex Index Type */
   private Type vertexIndexType;
-  /** Vertex Index Type */
+  /** Vertex Value Type */
   private Type vertexValueType;
-  /** Vertex Index Type */
+  /** Edge Value Type */
   private Type edgeValueType;
-  /** Vertex Index Type */
-  private Type messageValueType;
+  /** Outgoing Message Type */
+  private Type outgoingMessageValueType;
 
   /**
    * The Configuration object for use in the validation test.
@@ -110,14 +112,14 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
    */
   public void validateConfiguration() {
     checkConfiguration();
-    Class<? extends Vertex<I, V, E, M>> vertexClass =
-      conf.getVertexClass();
+    Class<? extends Computation<I, V, E, M1, M2>> computationClass =
+      conf.getComputationClass();
     List<Class<?>> classList = ReflectionUtils.getTypeArguments(
-      Vertex.class, vertexClass);
+      Computation.class, computationClass);
     vertexIndexType = classList.get(ID_PARAM_INDEX);
     vertexValueType = classList.get(VALUE_PARAM_INDEX);
     edgeValueType = classList.get(EDGE_PARAM_INDEX);
-    messageValueType = classList.get(MSG_PARAM_INDEX);
+    outgoingMessageValueType = classList.get(OUTGOING_MSG_PARAM_INDEX);
     verifyOutEdgesGenericTypes();
     verifyVertexInputFormatGenericTypes();
     verifyEdgeInputFormatGenericTypes();
@@ -146,9 +148,9 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
       throw new IllegalArgumentException("checkConfiguration: No valid " +
           GiraphConstants.MIN_WORKERS);
     }
-    if (conf.getVertexClass() == null) {
+    if (conf.getComputationClass() == null) {
       throw new IllegalArgumentException("checkConfiguration: Null " +
-          GiraphConstants.VERTEX_CLASS.getKey());
+          GiraphConstants.COMPUTATION_CLASS.getKey());
     }
     if (conf.getVertexInputFormatClass() == null &&
         conf.getEdgeInputFormatClass() == null) {
@@ -281,7 +283,7 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
 
   /** If there is a combiner type, verify its generic params match the job. */
   private void verifyVertexCombinerGenericTypes() {
-    Class<? extends Combiner<I, M>> vertexCombinerClass =
+    Class<? extends Combiner<I, M2>> vertexCombinerClass =
       conf.getCombinerClass();
     if (vertexCombinerClass != null) {
       List<Class<?>> classList =
@@ -293,10 +295,11 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
             "vertex - " + vertexIndexType +
             ", vertex combiner - " + classList.get(ID_PARAM_INDEX));
       }
-      if (!messageValueType.equals(classList.get(MSG_COMBINER_PARAM_INDEX))) {
+      if (!outgoingMessageValueType.equals(
+          classList.get(MSG_COMBINER_PARAM_INDEX))) {
         throw new IllegalArgumentException(
           "checkClassTypes: Message value types don't match, " +
-            "vertex - " + messageValueType +
+            "vertex - " + outgoingMessageValueType +
             ", vertex combiner - " + classList.get(MSG_COMBINER_PARAM_INDEX));
       }
     }
@@ -360,14 +363,14 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
   /** If there is a vertex resolver,
    * validate the generic parameter types. */
   private void verifyVertexResolverGenericTypes() {
-    Class<? extends VertexResolver<I, V, E, M>>
+    Class<? extends VertexResolver<I, V, E>>
       vrClass = conf.getVertexResolverClass();
     if (!DefaultVertexResolver.class.isAssignableFrom(vrClass)) {
       return;
     }
-    Class<? extends DefaultVertexResolver<I, V, E, M>>
+    Class<? extends DefaultVertexResolver<I, V, E>>
       dvrClass =
-        (Class<? extends DefaultVertexResolver<I, V, E, M>>) vrClass;
+        (Class<? extends DefaultVertexResolver<I, V, E>>) vrClass;
     List<Class<?>> classList =
       ReflectionUtils.getTypeArguments(
           DefaultVertexResolver.class, dvrClass);
@@ -392,13 +395,6 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
           "vertex - " + edgeValueType +
           ", vertex resolver - " + classList.get(EDGE_PARAM_INDEX));
     }
-    if (classList.get(MSG_PARAM_INDEX) != null &&
-      !messageValueType.equals(classList.get(MSG_PARAM_INDEX))) {
-      throw new IllegalArgumentException(
-        "checkClassTypes: Message value types don't match, " +
-          "vertex - " + messageValueType +
-          ", vertex resolver - " + classList.get(MSG_PARAM_INDEX));
-    }
   }
 }
 


[03/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
deleted file mode 100644
index 7a63e8d..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.List;
-import org.apache.giraph.aggregators.DoubleMaxAggregator;
-import org.apache.giraph.aggregators.DoubleMinAggregator;
-import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.VertexReader;
-import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
-import org.apache.giraph.io.formats.TextVertexOutputFormat;
-import org.apache.giraph.master.DefaultMasterCompute;
-import org.apache.giraph.worker.WorkerContext;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.log4j.Logger;
-
-/**
- * Demonstrates the basic Pregel PageRank implementation.
- */
-@Algorithm(
-    name = "Page rank"
-)
-public class SimplePageRankVertex extends Vertex<LongWritable,
-    DoubleWritable, FloatWritable, DoubleWritable> {
-  /** Number of supersteps for this test */
-  public static final int MAX_SUPERSTEPS = 30;
-  /** Logger */
-  private static final Logger LOG =
-      Logger.getLogger(SimplePageRankVertex.class);
-  /** Sum aggregator name */
-  private static String SUM_AGG = "sum";
-  /** Min aggregator name */
-  private static String MIN_AGG = "min";
-  /** Max aggregator name */
-  private static String MAX_AGG = "max";
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) {
-    if (getSuperstep() >= 1) {
-      double sum = 0;
-      for (DoubleWritable message : messages) {
-        sum += message.get();
-      }
-      DoubleWritable vertexValue =
-          new DoubleWritable((0.15f / getTotalNumVertices()) + 0.85f * sum);
-      setValue(vertexValue);
-      aggregate(MAX_AGG, vertexValue);
-      aggregate(MIN_AGG, vertexValue);
-      aggregate(SUM_AGG, new LongWritable(1));
-      LOG.info(getId() + ": PageRank=" + vertexValue +
-          " max=" + getAggregatedValue(MAX_AGG) +
-          " min=" + getAggregatedValue(MIN_AGG));
-    }
-
-    if (getSuperstep() < MAX_SUPERSTEPS) {
-      long edges = getNumEdges();
-      sendMessageToAllEdges(
-          new DoubleWritable(getValue().get() / edges));
-    } else {
-      voteToHalt();
-    }
-  }
-
-  /**
-   * Worker context used with {@link SimplePageRankVertex}.
-   */
-  public static class SimplePageRankVertexWorkerContext extends
-      WorkerContext {
-    /** Final max value for verification for local jobs */
-    private static double FINAL_MAX;
-    /** Final min value for verification for local jobs */
-    private static double FINAL_MIN;
-    /** Final sum value for verification for local jobs */
-    private static long FINAL_SUM;
-
-    public static double getFinalMax() {
-      return FINAL_MAX;
-    }
-
-    public static double getFinalMin() {
-      return FINAL_MIN;
-    }
-
-    public static long getFinalSum() {
-      return FINAL_SUM;
-    }
-
-    @Override
-    public void preApplication()
-      throws InstantiationException, IllegalAccessException {
-    }
-
-    @Override
-    public void postApplication() {
-      FINAL_SUM = this.<LongWritable>getAggregatedValue(SUM_AGG).get();
-      FINAL_MAX = this.<DoubleWritable>getAggregatedValue(MAX_AGG).get();
-      FINAL_MIN = this.<DoubleWritable>getAggregatedValue(MIN_AGG).get();
-
-      LOG.info("aggregatedNumVertices=" + FINAL_SUM);
-      LOG.info("aggregatedMaxPageRank=" + FINAL_MAX);
-      LOG.info("aggregatedMinPageRank=" + FINAL_MIN);
-    }
-
-    @Override
-    public void preSuperstep() {
-      if (getSuperstep() >= 3) {
-        LOG.info("aggregatedNumVertices=" +
-            getAggregatedValue(SUM_AGG) +
-            " NumVertices=" + getTotalNumVertices());
-        if (this.<LongWritable>getAggregatedValue(SUM_AGG).get() !=
-            getTotalNumVertices()) {
-          throw new RuntimeException("wrong value of SumAggreg: " +
-              getAggregatedValue(SUM_AGG) + ", should be: " +
-              getTotalNumVertices());
-        }
-        DoubleWritable maxPagerank = getAggregatedValue(MAX_AGG);
-        LOG.info("aggregatedMaxPageRank=" + maxPagerank.get());
-        DoubleWritable minPagerank = getAggregatedValue(MIN_AGG);
-        LOG.info("aggregatedMinPageRank=" + minPagerank.get());
-      }
-    }
-
-    @Override
-    public void postSuperstep() { }
-  }
-
-  /**
-   * Master compute associated with {@link SimplePageRankVertex}.
-   * It registers required aggregators.
-   */
-  public static class SimplePageRankVertexMasterCompute extends
-      DefaultMasterCompute {
-    @Override
-    public void initialize() throws InstantiationException,
-        IllegalAccessException {
-      registerAggregator(SUM_AGG, LongSumAggregator.class);
-      registerPersistentAggregator(MIN_AGG, DoubleMinAggregator.class);
-      registerPersistentAggregator(MAX_AGG, DoubleMaxAggregator.class);
-    }
-  }
-
-  /**
-   * Simple VertexReader that supports {@link SimplePageRankVertex}
-   */
-  public static class SimplePageRankVertexReader extends
-      GeneratedVertexReader<LongWritable, DoubleWritable, FloatWritable> {
-    /** Class logger */
-    private static final Logger LOG =
-        Logger.getLogger(SimplePageRankVertexReader.class);
-
-    @Override
-    public boolean nextVertex() {
-      return totalRecords > recordsRead;
-    }
-
-    @Override
-    public Vertex<LongWritable, DoubleWritable,
-        FloatWritable, Writable> getCurrentVertex() throws IOException {
-      Vertex<LongWritable, DoubleWritable, FloatWritable, Writable>
-          vertex = getConf().createVertex();
-      LongWritable vertexId = new LongWritable(
-          (inputSplit.getSplitIndex() * totalRecords) + recordsRead);
-      DoubleWritable vertexValue = new DoubleWritable(vertexId.get() * 10d);
-      long targetVertexId =
-          (vertexId.get() + 1) %
-          (inputSplit.getNumSplits() * totalRecords);
-      float edgeValue = vertexId.get() * 100f;
-      List<Edge<LongWritable, FloatWritable>> edges = Lists.newLinkedList();
-      edges.add(EdgeFactory.create(new LongWritable(targetVertexId),
-          new FloatWritable(edgeValue)));
-      vertex.initialize(vertexId, vertexValue, edges);
-      ++recordsRead;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("next: Return vertexId=" + vertex.getId().get() +
-            ", vertexValue=" + vertex.getValue() +
-            ", targetVertexId=" + targetVertexId + ", edgeValue=" + edgeValue);
-      }
-      return vertex;
-    }
-  }
-
-  /**
-   * Simple VertexInputFormat that supports {@link SimplePageRankVertex}
-   */
-  public static class SimplePageRankVertexInputFormat extends
-    GeneratedVertexInputFormat<LongWritable, DoubleWritable, FloatWritable> {
-    @Override
-    public VertexReader<LongWritable, DoubleWritable,
-    FloatWritable> createVertexReader(InputSplit split,
-      TaskAttemptContext context)
-      throws IOException {
-      return new SimplePageRankVertexReader();
-    }
-  }
-
-  /**
-   * Simple VertexOutputFormat that supports {@link SimplePageRankVertex}
-   */
-  public static class SimplePageRankVertexOutputFormat extends
-      TextVertexOutputFormat<LongWritable, DoubleWritable, FloatWritable> {
-    @Override
-    public TextVertexWriter createVertexWriter(TaskAttemptContext context)
-      throws IOException, InterruptedException {
-      return new SimplePageRankVertexWriter();
-    }
-
-    /**
-     * Simple VertexWriter that supports {@link SimplePageRankVertex}
-     */
-    public class SimplePageRankVertexWriter extends TextVertexWriter {
-      @Override
-      public void writeVertex(
-          Vertex<LongWritable, DoubleWritable, FloatWritable, ?> vertex)
-        throws IOException, InterruptedException {
-        getRecordWriter().write(
-            new Text(vertex.getId().toString()),
-            new Text(vertex.getValue().toString()));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsComputation.java
new file mode 100644
index 0000000..bc39cad
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsComputation.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.conf.LongConfOption;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Demonstrates the basic Pregel shortest paths implementation.
+ */
+@Algorithm(
+    name = "Shortest paths",
+    description = "Finds all shortest paths from a selected vertex"
+)
+public class SimpleShortestPathsComputation extends BasicComputation<
+    LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+  /** The shortest paths id */
+  public static final LongConfOption SOURCE_ID =
+      new LongConfOption("SimpleShortestPathsVertex.sourceId", 1);
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(SimpleShortestPathsComputation.class);
+
+  /**
+   * Is this vertex the source id?
+   *
+   * @param vertex Vertex
+   * @return True if the source id
+   */
+  private boolean isSource(Vertex<LongWritable, ?, ?> vertex) {
+    return vertex.getId().get() == SOURCE_ID.get(getConf());
+  }
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    if (getSuperstep() == 0) {
+      vertex.setValue(new DoubleWritable(Double.MAX_VALUE));
+    }
+    double minDist = isSource(vertex) ? 0d : Double.MAX_VALUE;
+    for (DoubleWritable message : messages) {
+      minDist = Math.min(minDist, message.get());
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Vertex " + vertex.getId() + " got minDist = " + minDist +
+          " vertex value = " + vertex.getValue());
+    }
+    if (minDist < vertex.getValue().get()) {
+      vertex.setValue(new DoubleWritable(minDist));
+      for (Edge<LongWritable, FloatWritable> edge : vertex.getEdges()) {
+        double distance = minDist + edge.getValue().get();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Vertex " + vertex.getId() + " sent to " +
+              edge.getTargetVertexId() + " = " + distance);
+        }
+        sendMessage(edge.getTargetVertexId(), new DoubleWritable(distance));
+      }
+    }
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
deleted file mode 100644
index 13d1d7c..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.conf.LongConfOption;
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.log4j.Logger;
-
-/**
- * Demonstrates the basic Pregel shortest paths implementation.
- */
-@Algorithm(
-    name = "Shortest paths",
-    description = "Finds all shortest paths from a selected vertex"
-)
-public class SimpleShortestPathsVertex extends
-    Vertex<LongWritable, DoubleWritable,
-    FloatWritable, DoubleWritable> {
-  /** The shortest paths id */
-  public static final LongConfOption SOURCE_ID =
-      new LongConfOption("SimpleShortestPathsVertex.sourceId", 1);
-  /** Class logger */
-  private static final Logger LOG =
-      Logger.getLogger(SimpleShortestPathsVertex.class);
-
-  /**
-   * Is this vertex the source id?
-   *
-   * @return True if the source id
-   */
-  private boolean isSource() {
-    return getId().get() == SOURCE_ID.get(getConf());
-  }
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) {
-    if (getSuperstep() == 0) {
-      setValue(new DoubleWritable(Double.MAX_VALUE));
-    }
-    double minDist = isSource() ? 0d : Double.MAX_VALUE;
-    for (DoubleWritable message : messages) {
-      minDist = Math.min(minDist, message.get());
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Vertex " + getId() + " got minDist = " + minDist +
-          " vertex value = " + getValue());
-    }
-    if (minDist < getValue().get()) {
-      setValue(new DoubleWritable(minDist));
-      for (Edge<LongWritable, FloatWritable> edge : getEdges()) {
-        double distance = minDist + edge.getValue().get();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Vertex " + getId() + " sent to " +
-              edge.getTargetVertexId() + " = " + distance);
-        }
-        sendMessage(edge.getTargetVertexId(), new DoubleWritable(distance));
-      }
-    }
-    voteToHalt();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java
new file mode 100644
index 0000000..c3fd215
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.VertexReader;
+import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.log4j.Logger;
+
+/**
+ * Just a simple Vertex compute implementation that executes 3 supersteps, then
+ * finishes.
+ */
+public class SimpleSuperstepComputation extends BasicComputation<LongWritable,
+    IntWritable, FloatWritable, IntWritable> {
+  @Override
+  public void compute(
+      Vertex<LongWritable, IntWritable, FloatWritable> vertex,
+      Iterable<IntWritable> messages) throws IOException {
+    // Some checks for additional testing
+    if (getTotalNumVertices() < 1) {
+      throw new IllegalStateException("compute: Illegal total vertices " +
+          getTotalNumVertices());
+    }
+    if (getTotalNumEdges() < 0) {
+      throw new IllegalStateException("compute: Illegal total edges " +
+          getTotalNumEdges());
+    }
+    if (vertex.isHalted()) {
+      throw new IllegalStateException("compute: Impossible to be halted - " +
+          vertex.isHalted());
+    }
+
+    if (getSuperstep() > 3) {
+      vertex.voteToHalt();
+    }
+  }
+
+  /**
+   * Simple VertexReader that supports {@link SimpleSuperstepComputation}
+   */
+  public static class SimpleSuperstepVertexReader extends
+      GeneratedVertexReader<LongWritable, IntWritable, FloatWritable> {
+    /** Class logger */
+    private static final Logger LOG =
+        Logger.getLogger(SimpleSuperstepVertexReader.class);
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+      return totalRecords > recordsRead;
+    }
+
+    @Override
+    public Vertex<LongWritable, IntWritable, FloatWritable> getCurrentVertex()
+      throws IOException, InterruptedException {
+      Vertex<LongWritable, IntWritable, FloatWritable> vertex =
+          getConf().createVertex();
+      long tmpId = reverseIdOrder ?
+          ((inputSplit.getSplitIndex() + 1) * totalRecords) -
+          recordsRead - 1 :
+            (inputSplit.getSplitIndex() * totalRecords) + recordsRead;
+      LongWritable vertexId = new LongWritable(tmpId);
+      IntWritable vertexValue =
+          new IntWritable((int) (vertexId.get() * 10));
+      List<Edge<LongWritable, FloatWritable>> edges = Lists.newLinkedList();
+      long targetVertexId =
+          (vertexId.get() + 1) %
+          (inputSplit.getNumSplits() * totalRecords);
+      float edgeValue = vertexId.get() * 100f;
+      edges.add(EdgeFactory.create(new LongWritable(targetVertexId),
+          new FloatWritable(edgeValue)));
+      vertex.initialize(vertexId, vertexValue, edges);
+      ++recordsRead;
+      if (LOG.isInfoEnabled()) {
+        LOG.info("next: Return vertexId=" + vertex.getId().get() +
+            ", vertexValue=" + vertex.getValue() +
+            ", targetVertexId=" + targetVertexId +
+            ", edgeValue=" + edgeValue);
+      }
+      return vertex;
+    }
+  }
+
+  /**
+   * Simple VertexInputFormat that supports {@link SimpleSuperstepComputation}
+   */
+  public static class SimpleSuperstepVertexInputFormat extends
+    GeneratedVertexInputFormat<LongWritable, IntWritable, FloatWritable> {
+    @Override
+    public VertexReader<LongWritable, IntWritable, FloatWritable>
+    createVertexReader(InputSplit split, TaskAttemptContext context)
+      throws IOException {
+      return new SimpleSuperstepVertexReader();
+    }
+  }
+
+
+  /**
+   * Simple VertexOutputFormat that supports {@link SimpleSuperstepComputation}
+   */
+  public static class SimpleSuperstepVertexOutputFormat extends
+      TextVertexOutputFormat<LongWritable, IntWritable, FloatWritable> {
+    @Override
+    public TextVertexWriter createVertexWriter(TaskAttemptContext context)
+      throws IOException, InterruptedException {
+      return new SimpleSuperstepVertexWriter();
+    }
+
+    /**
+     * Simple VertexWriter that supports {@link SimpleSuperstepComputation}
+     */
+    public class SimpleSuperstepVertexWriter extends TextVertexWriter {
+      @Override
+      public void writeVertex(Vertex<LongWritable, IntWritable,
+          FloatWritable> vertex) throws IOException, InterruptedException {
+        getRecordWriter().write(
+            new Text(vertex.getId().toString()),
+            new Text(vertex.getValue().toString()));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
deleted file mode 100644
index 6f8b352..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.List;
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.VertexReader;
-import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
-import org.apache.giraph.io.formats.TextVertexOutputFormat;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.log4j.Logger;
-
-/**
- * Just a simple Vertex compute implementation that executes 3 supersteps, then
- * finishes.
- */
-public class SimpleSuperstepVertex extends Vertex<LongWritable, IntWritable,
-    FloatWritable, IntWritable> {
-  @Override
-  public void compute(Iterable<IntWritable> messages) {
-    // Some checks for additional testing
-    if (getTotalNumVertices() < 1) {
-      throw new IllegalStateException("compute: Illegal total vertices " +
-          getTotalNumVertices());
-    }
-    if (getTotalNumEdges() < 0) {
-      throw new IllegalStateException("compute: Illegal total edges " +
-          getTotalNumEdges());
-    }
-    if (isHalted()) {
-      throw new IllegalStateException("compute: Impossible to be halted - " +
-          isHalted());
-    }
-
-    if (getSuperstep() > 3) {
-      voteToHalt();
-    }
-  }
-
-  /**
-   * Simple VertexReader that supports {@link SimpleSuperstepVertex}
-   */
-  public static class SimpleSuperstepVertexReader extends
-      GeneratedVertexReader<LongWritable, IntWritable, FloatWritable> {
-    /** Class logger */
-    private static final Logger LOG =
-        Logger.getLogger(SimpleSuperstepVertexReader.class);
-
-    @Override
-    public boolean nextVertex() throws IOException, InterruptedException {
-      return totalRecords > recordsRead;
-    }
-
-    @Override
-    public Vertex<LongWritable, IntWritable, FloatWritable,
-        Writable> getCurrentVertex()
-      throws IOException, InterruptedException {
-      Vertex<LongWritable, IntWritable, FloatWritable, Writable> vertex =
-          getConf().createVertex();
-      long tmpId = reverseIdOrder ?
-          ((inputSplit.getSplitIndex() + 1) * totalRecords) -
-          recordsRead - 1 :
-            (inputSplit.getSplitIndex() * totalRecords) + recordsRead;
-      LongWritable vertexId = new LongWritable(tmpId);
-      IntWritable vertexValue =
-          new IntWritable((int) (vertexId.get() * 10));
-      List<Edge<LongWritable, FloatWritable>> edges = Lists.newLinkedList();
-      long targetVertexId =
-          (vertexId.get() + 1) %
-          (inputSplit.getNumSplits() * totalRecords);
-      float edgeValue = vertexId.get() * 100f;
-      edges.add(EdgeFactory.create(new LongWritable(targetVertexId),
-          new FloatWritable(edgeValue)));
-      vertex.initialize(vertexId, vertexValue, edges);
-      ++recordsRead;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("next: Return vertexId=" + vertex.getId().get() +
-            ", vertexValue=" + vertex.getValue() +
-            ", targetVertexId=" + targetVertexId +
-            ", edgeValue=" + edgeValue);
-      }
-      return vertex;
-    }
-  }
-
-  /**
-   * Simple VertexInputFormat that supports {@link SimpleSuperstepVertex}
-   */
-  public static class SimpleSuperstepVertexInputFormat extends
-    GeneratedVertexInputFormat<LongWritable, IntWritable, FloatWritable> {
-    @Override
-    public VertexReader<LongWritable, IntWritable, FloatWritable>
-    createVertexReader(InputSplit split, TaskAttemptContext context)
-      throws IOException {
-      return new SimpleSuperstepVertexReader();
-    }
-  }
-
-
-  /**
-   * Simple VertexOutputFormat that supports {@link SimpleSuperstepVertex}
-   */
-  public static class SimpleSuperstepVertexOutputFormat extends
-      TextVertexOutputFormat<LongWritable, IntWritable, FloatWritable> {
-    @Override
-    public TextVertexWriter createVertexWriter(TaskAttemptContext context)
-      throws IOException, InterruptedException {
-      return new SimpleSuperstepVertexWriter();
-    }
-
-    /**
-     * Simple VertexWriter that supports {@link SimpleSuperstepVertex}
-     */
-    public class SimpleSuperstepVertexWriter extends TextVertexWriter {
-      @Override
-      public void writeVertex(Vertex<LongWritable, IntWritable,
-          FloatWritable, ?> vertex) throws IOException, InterruptedException {
-        getRecordWriter().write(
-            new Text(vertex.getId().toString()),
-            new Text(vertex.getValue().toString()));
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
index 157e6ef..b1ee2a0 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTextVertexOutputFormat.java
@@ -39,7 +39,7 @@ public class SimpleTextVertexOutputFormat extends
   private class SimpleTextVertexWriter extends TextVertexWriter {
     @Override
     public void writeVertex(
-      Vertex<LongWritable, IntWritable, FloatWritable, ?> vertex)
+      Vertex<LongWritable, IntWritable, FloatWritable> vertex)
       throws IOException, InterruptedException {
       getRecordWriter().write(
           new Text(vertex.getId().toString()),

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingComputation.java
new file mode 100644
index 0000000..8608d02
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingComputation.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.utils.ArrayListWritable;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Demonstrates triangle closing in simple,
+ * unweighted graphs for Giraph.
+ *
+ * Triangle Closing: Vertex A and B maintain out-edges to C and D
+ * The algorithm, when finished, populates all vertices' value with an
+ * array of Writables representing all the vertices that each
+ * should form an out-edge to (connect with, if this is a social
+ * graph.)
+ * In this example, vertices A and B would hold empty arrays
+ * since they are already connected with C and D. Results:
+ * If the graph is undirected, C would hold value, D and D would
+ * hold value C, since both are neighbors of A and B and yet both
+ * were not previously connected to each other.
+ *
+ * In a social graph, the result values for vertex X would represent people
+ * that are likely a part of a person X's social circle (they know one or more
+ * people X is connected to already) but X had not previously met them yet.
+ * Given this new information, X can decide to connect to vertices (peoople) in
+ * the result array or not.
+ *
+ * Results at each vertex are ordered in terms of the # of neighbors
+ * who are connected to each vertex listed in the final vertex value.
+ * The more of a vertex's neighbors who "know" someone, the stronger
+ * your social relationship is presumed to be to that vertex (assuming
+ * a social graph) and the more likely you should connect with them.
+ *
+ * In this implementation, Edge Values are not used, but could be
+ * adapted to represent additional qualities that could affect the
+ * ordering of the final result array.
+ */
+public class SimpleTriangleClosingComputation extends BasicComputation<
+  IntWritable, SimpleTriangleClosingComputation.IntArrayListWritable,
+  NullWritable, IntWritable> {
+  /** Vertices to close the triangle, ranked by frequency of in-msgs */
+  private Map<IntWritable, Integer> closeMap =
+    Maps.<IntWritable, Integer>newHashMap();
+
+  @Override
+  public void compute(
+      Vertex<IntWritable, IntArrayListWritable, NullWritable> vertex,
+      Iterable<IntWritable> messages) throws IOException {
+    if (getSuperstep() == 0) {
+      // send list of this vertex's neighbors to all neighbors
+      for (Edge<IntWritable, NullWritable> edge : vertex.getEdges()) {
+        sendMessageToAllEdges(vertex, edge.getTargetVertexId());
+      }
+    } else {
+      for (IntWritable message : messages) {
+        final int current = (closeMap.get(message) == null) ?
+          0 : closeMap.get(message) + 1;
+        closeMap.put(message, current);
+      }
+      // make sure the result values are sorted and
+      // packaged in an IntArrayListWritable for output
+      Set<Pair> sortedResults = Sets.<Pair>newTreeSet();
+      for (Map.Entry<IntWritable, Integer> entry : closeMap.entrySet()) {
+        sortedResults.add(new Pair(entry.getKey(), entry.getValue()));
+      }
+      IntArrayListWritable
+        outputList = new IntArrayListWritable();
+      for (Pair pair : sortedResults) {
+        if (pair.value > 0) {
+          outputList.add(pair.key);
+        } else {
+          break;
+        }
+      }
+      vertex.setValue(outputList);
+    }
+    vertex.voteToHalt();
+  }
+
+  /** Quick, immutable K,V storage for sorting in tree set */
+  public static class Pair implements Comparable<Pair> {
+    /** key
+     * @param key the IntWritable key */
+    private final IntWritable key;
+    /** value
+     * @param value the Integer value */
+    private final Integer value;
+    /** Constructor
+     * @param k the key
+     * @param v the value
+     */
+    public Pair(IntWritable k, Integer v) {
+      key = k;
+      value = v;
+    }
+    /** key getter
+     * @return the key */
+    public IntWritable getKey() { return key; }
+    /** value getter
+     * @return the value */
+    public Integer getValue() { return value; }
+    /** Comparator to quickly sort by values
+     * @param other the Pair to compare with THIS
+     * @return the comparison value as an integer */
+    @Override
+    public int compareTo(Pair other) {
+      return other.value - this.value;
+    }
+  }
+
+  /** Utility class for delivering the array of vertices THIS vertex
+    * should connect with to close triangles with neighbors */
+  public static class IntArrayListWritable
+    extends ArrayListWritable<IntWritable> {
+    /** Default constructor for reflection */
+    public IntArrayListWritable() {
+      super();
+    }
+    /** Set storage type for this ArrayListWritable */
+    @Override
+    @SuppressWarnings("unchecked")
+    public void setClass() {
+      setClass(IntWritable.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java
deleted file mode 100644
index f44cb18..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleTriangleClosingVertex.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.utils.ArrayListWritable;
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Demonstrates triangle closing in simple,
- * unweighted graphs for Giraph.
- *
- * Triangle Closing: Vertex A and B maintain out-edges to C and D
- * The algorithm, when finished, populates all vertices' value with an
- * array of Writables representing all the vertices that each
- * should form an out-edge to (connect with, if this is a social
- * graph.)
- * In this example, vertices A and B would hold empty arrays
- * since they are already connected with C and D. Results:
- * If the graph is undirected, C would hold value, D and D would
- * hold value C, since both are neighbors of A and B and yet both
- * were not previously connected to each other.
- *
- * In a social graph, the result values for vertex X would represent people
- * that are likely a part of a person X's social circle (they know one or more
- * people X is connected to already) but X had not previously met them yet.
- * Given this new information, X can decide to connect to vertices (peoople) in
- * the result array or not.
- *
- * Results at each vertex are ordered in terms of the # of neighbors
- * who are connected to each vertex listed in the final vertex value.
- * The more of a vertex's neighbors who "know" someone, the stronger
- * your social relationship is presumed to be to that vertex (assuming
- * a social graph) and the more likely you should connect with them.
- *
- * In this implementation, Edge Values are not used, but could be
- * adapted to represent additional qualities that could affect the
- * ordering of the final result array.
- */
-public class SimpleTriangleClosingVertex extends Vertex<
-  IntWritable, SimpleTriangleClosingVertex.IntArrayListWritable,
-  NullWritable, IntWritable> {
-  /** Vertices to close the triangle, ranked by frequency of in-msgs */
-  private Map<IntWritable, Integer> closeMap =
-    Maps.<IntWritable, Integer>newHashMap();
-
-  @Override
-  public void compute(Iterable<IntWritable> messages) {
-    if (getSuperstep() == 0) {
-      // send list of this vertex's neighbors to all neighbors
-      for (Edge<IntWritable, NullWritable> edge : getEdges()) {
-        sendMessageToAllEdges(edge.getTargetVertexId());
-      }
-    } else {
-      for (IntWritable message : messages) {
-        final int current = (closeMap.get(message) == null) ?
-          0 : closeMap.get(message) + 1;
-        closeMap.put(message, current);
-      }
-      // make sure the result values are sorted and
-      // packaged in an IntArrayListWritable for output
-      Set<SimpleTriangleClosingVertex.Pair> sortedResults =
-        Sets.<SimpleTriangleClosingVertex.Pair>newTreeSet();
-      for (Map.Entry<IntWritable, Integer> entry : closeMap.entrySet()) {
-        sortedResults.add(new Pair(entry.getKey(), entry.getValue()));
-      }
-      SimpleTriangleClosingVertex.IntArrayListWritable
-        outputList = new SimpleTriangleClosingVertex.IntArrayListWritable();
-      for (SimpleTriangleClosingVertex.Pair pair : sortedResults) {
-        if (pair.value > 0) {
-          outputList.add(pair.key);
-        } else {
-          break;
-        }
-      }
-      setValue(outputList);
-    }
-    voteToHalt();
-  }
-
-  /** Quick, immutable K,V storage for sorting in tree set */
-  public static class Pair implements Comparable<Pair> {
-    /** key
-     * @param key the IntWritable key */
-    private final IntWritable key;
-    /** value
-     * @param value the Integer value */
-    private final Integer value;
-    /** Constructor
-     * @param k the key
-     * @param v the value
-     */
-    public Pair(IntWritable k, Integer v) {
-      key = k;
-      value = v;
-    }
-    /** key getter
-     * @return the key */
-    public IntWritable getKey() { return key; }
-    /** value getter
-     * @return the value */
-    public Integer getValue() { return value; }
-    /** Comparator to quickly sort by values
-     * @param other the Pair to compare with THIS
-     * @return the comparison value as an integer */
-    @Override
-    public int compareTo(Pair other) {
-      return other.value - this.value;
-    }
-  }
-
-  /** Utility class for delivering the array of vertices THIS vertex
-    * should connect with to close triangles with neighbors */
-  public static class IntArrayListWritable
-    extends ArrayListWritable<IntWritable> {
-    /** Default constructor for reflection */
-    public IntArrayListWritable() {
-      super();
-    }
-    /** Set storage type for this ArrayListWritable */
-    @Override
-    @SuppressWarnings("unchecked")
-    public void setClass() {
-      setClass(IntWritable.class);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
index 8a6f775..a5051b8 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleVertexWithWorkerContext.java
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.examples.SimpleSuperstepVertex.
-    SimpleSuperstepVertexInputFormat;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.worker.WorkerContext;
@@ -63,21 +63,21 @@ public class SimpleVertexWithWorkerContext implements Tool {
   /**
    * Actual vetex implementation
    */
-  public static class SimpleVertex extends
-      Vertex<LongWritable, IntWritable, FloatWritable,
-          DoubleWritable> {
+  public static class SimpleComputation extends BasicComputation<LongWritable,
+      IntWritable, FloatWritable, DoubleWritable> {
     @Override
-    public void compute(Iterable<DoubleWritable> messages) throws IOException {
+    public void compute(
+        Vertex<LongWritable, IntWritable, FloatWritable> vertex,
+        Iterable<DoubleWritable> messages) throws IOException {
 
       long superstep = getSuperstep();
 
       if (superstep < TESTLENGTH) {
-        EmitterWorkerContext emitter =
-            (EmitterWorkerContext) getWorkerContext();
-        emitter.emit("vertexId=" + getId() +
+        EmitterWorkerContext emitter = getWorkerContext();
+        emitter.emit("vertexId=" + vertex.getId() +
             " superstep=" + superstep + "\n");
       } else {
-        voteToHalt();
+        vertex.voteToHalt();
       }
     }
   }
@@ -169,7 +169,7 @@ public class SimpleVertexWithWorkerContext implements Tool {
           "run: Must have 2 arguments <output path> <# of workers>");
     }
     GiraphJob job = new GiraphJob(getConf(), getClass().getName());
-    job.getConfiguration().setVertexClass(SimpleVertex.class);
+    job.getConfiguration().setComputationClass(SimpleComputation.class);
     job.getConfiguration().setVertexInputFormatClass(
         SimpleSuperstepVertexInputFormat.class);
     job.getConfiguration().setWorkerContextClass(EmitterWorkerContext.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/TestComputationStateComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/TestComputationStateComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/TestComputationStateComputation.java
new file mode 100644
index 0000000..ad72951
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/TestComputationStateComputation.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.worker.DefaultWorkerContext;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Vertex to test the local variables in Computation, and pre/postSuperstep
+ * methods
+ */
+public class TestComputationStateComputation extends BasicComputation<
+    LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+  /** How many compute threads to use in the test */
+  public static final int NUM_COMPUTE_THREADS = 10;
+  /** How many vertices to create for the test */
+  public static final int NUM_VERTICES = 100;
+  /** How many partitions to have */
+  public static final int NUM_PARTITIONS = 25;
+
+  /**
+   * The counter should hold the number of vertices in this partition,
+   * plus the current superstep
+   */
+  private long counter;
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    counter++;
+    if (getSuperstep() > 5) {
+      vertex.voteToHalt();
+    }
+  }
+
+  @Override
+  public void preSuperstep() {
+    counter =
+      ((TestComputationStateWorkerContext) getWorkerContext()).superstepCounter;
+  }
+
+  @Override
+  public void postSuperstep() {
+    ((TestComputationStateWorkerContext) getWorkerContext()).totalCounter
+        .addAndGet(counter);
+  }
+
+  /**
+   * WorkerContext for TestComputationState
+   */
+  public static class TestComputationStateWorkerContext extends
+      DefaultWorkerContext {
+    /** Current superstep */
+    private long superstepCounter;
+    /**
+     * This counter should hold the sum of Computation's counters
+     */
+    private AtomicLong totalCounter;
+
+    @Override
+    public void preSuperstep() {
+      superstepCounter = getSuperstep();
+      totalCounter = new AtomicLong(0);
+    }
+
+    @Override
+    public void postSuperstep() {
+      assertEquals(totalCounter.get(),
+          NUM_PARTITIONS * superstepCounter + getTotalNumVertices());
+    }
+  }
+
+  /**
+   * Throws exception if values are not equal.
+   *
+   * @param expected Expected value
+   * @param actual   Actual value
+   */
+  private static void assertEquals(long expected, long actual) {
+    if (expected != actual) {
+      throw new RuntimeException("expected: " + expected +
+          ", actual: " + actual);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java b/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
index 6e3c589..bcb02cf 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
@@ -19,6 +19,7 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.master.DefaultMasterCompute;
@@ -41,7 +42,7 @@ import java.io.IOException;
  */
 public class VerifyMessage {
   /**
-   * Message that will be sent in {@link VerifyMessageVertex}.
+   * Message that will be sent in {@link VerifyMessageComputation}.
    */
   public static class VerifiableMessage implements Writable {
     /** Superstep sent on */
@@ -93,9 +94,9 @@ public class VerifyMessage {
   /**
    * Send and verify messages.
    */
-  public static class VerifyMessageVertex extends
-      Vertex<LongWritable, IntWritable, FloatWritable,
-      VerifiableMessage> {
+  public static class VerifyMessageComputation extends
+      BasicComputation<LongWritable, IntWritable, FloatWritable,
+          VerifiableMessage> {
     /** Dynamically set number of SUPERSTEPS */
     public static final String SUPERSTEP_COUNT =
         "verifyMessageVertex.superstepCount";
@@ -104,14 +105,15 @@ public class VerifyMessage {
     /** Number of SUPERSTEPS to run (6 by default) */
     private static int SUPERSTEPS = 6;
     /** Class logger */
-    private static Logger LOG = Logger.getLogger(VerifyMessageVertex.class);
+    private static Logger LOG =
+        Logger.getLogger(VerifyMessageComputation.class);
 
     public static long getFinalSum() {
       return FINAL_SUM;
     }
 
     /**
-     * Worker context used with {@link VerifyMessageVertex}.
+     * Worker context used with {@link VerifyMessageComputation}.
      */
     public static class VerifyMessageVertexWorkerContext extends
         WorkerContext {
@@ -138,28 +140,30 @@ public class VerifyMessage {
     }
 
     @Override
-    public void compute(Iterable<VerifiableMessage> messages) {
+    public void compute(
+        Vertex<LongWritable, IntWritable, FloatWritable> vertex,
+        Iterable<VerifiableMessage> messages) throws IOException {
       String sumAggregatorName = LongSumAggregator.class.getName();
       if (getSuperstep() > SUPERSTEPS) {
-        voteToHalt();
+        vertex.voteToHalt();
         return;
       }
       if (LOG.isDebugEnabled()) {
         LOG.debug("compute: " + getAggregatedValue(sumAggregatorName));
       }
-      aggregate(sumAggregatorName, new LongWritable(getId().get()));
+      aggregate(sumAggregatorName, new LongWritable(vertex.getId().get()));
       if (LOG.isDebugEnabled()) {
         LOG.debug("compute: sum = " +
             this.<LongWritable>getAggregatedValue(sumAggregatorName).get() +
-            " for vertex " + getId());
+            " for vertex " + vertex.getId());
       }
       float msgValue = 0.0f;
       for (VerifiableMessage message : messages) {
         msgValue += message.value;
         if (LOG.isDebugEnabled()) {
           LOG.debug("compute: got msg = " + message +
-              " for vertex id " + getId() +
-              ", vertex value " + getValue() +
+              " for vertex id " + vertex.getId() +
+              ", vertex value " + vertex.getValue() +
               " on superstep " + getSuperstep());
         }
         if (message.superstep != getSuperstep() - 1) {
@@ -168,44 +172,44 @@ public class VerifyMessage {
                   "the previous superstep, current superstep = " +
                   getSuperstep());
         }
-        if ((message.sourceVertexId != getId().get() - 1) &&
-            (getId().get() != 0)) {
+        if ((message.sourceVertexId != vertex.getId().get() - 1) &&
+            (vertex.getId().get() != 0)) {
           throw new IllegalStateException(
               "compute: Impossible that this message didn't come " +
                   "from the previous vertex and came from " +
                   message.sourceVertexId);
         }
       }
-      int vertexValue = getValue().get();
-      setValue(new IntWritable(vertexValue + (int) msgValue));
+      int vertexValue = vertex.getValue().get();
+      vertex.setValue(new IntWritable(vertexValue + (int) msgValue));
       if (LOG.isDebugEnabled()) {
-        LOG.debug("compute: vertex " + getId() +
-            " has value " + getValue() +
+        LOG.debug("compute: vertex " + vertex.getId() +
+            " has value " + vertex.getValue() +
             " on superstep " + getSuperstep());
       }
-      for (Edge<LongWritable, FloatWritable> edge : getEdges()) {
+      for (Edge<LongWritable, FloatWritable> edge : vertex.getEdges()) {
         FloatWritable newEdgeValue = new FloatWritable(
             edge.getValue().get() + (float) vertexValue);
         Edge<LongWritable, FloatWritable> newEdge =
             EdgeFactory.create(edge.getTargetVertexId(), newEdgeValue);
         if (LOG.isDebugEnabled()) {
-          LOG.debug("compute: vertex " + getId() +
+          LOG.debug("compute: vertex " + vertex.getId() +
               " sending edgeValue " + edge.getValue() +
               " vertexValue " + vertexValue +
               " total " + newEdgeValue +
               " to vertex " + edge.getTargetVertexId() +
               " on superstep " + getSuperstep());
         }
-        addEdge(newEdge);
+        vertex.addEdge(newEdge);
         sendMessage(edge.getTargetVertexId(),
             new VerifiableMessage(
-                getSuperstep(), getId().get(), newEdgeValue.get()));
+                getSuperstep(), vertex.getId().get(), newEdgeValue.get()));
       }
     }
   }
 
   /**
-   * Master compute associated with {@link VerifyMessageVertex}.
+   * Master compute associated with {@link VerifyMessageComputation}.
    * It registers required aggregators.
    */
   public static class VerifyMessageMasterCompute extends

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueDoubleEdgeTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueDoubleEdgeTextOutputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueDoubleEdgeTextOutputFormat.java
index d328153..3d06561 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueDoubleEdgeTextOutputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueDoubleEdgeTextOutputFormat.java
@@ -44,7 +44,7 @@ public class VertexWithDoubleValueDoubleEdgeTextOutputFormat extends
   public class VertexWithDoubleValueWriter extends TextVertexWriter {
     @Override
     public void writeVertex(
-      Vertex<LongWritable, DoubleWritable, DoubleWritable, ?> vertex)
+      Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex)
       throws IOException, InterruptedException {
       StringBuilder output = new StringBuilder();
       output.append(vertex.getId().get());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueNullEdgeTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueNullEdgeTextOutputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueNullEdgeTextOutputFormat.java
index 85f3556..8cc769f 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueNullEdgeTextOutputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/VertexWithDoubleValueNullEdgeTextOutputFormat.java
@@ -47,7 +47,7 @@ public class VertexWithDoubleValueNullEdgeTextOutputFormat extends
   public class VertexWithDoubleValueWriter extends TextVertexWriter {
     @Override
     public void writeVertex(
-        Vertex<LongWritable, DoubleWritable, NullWritable, ?> vertex)
+        Vertex<LongWritable, DoubleWritable, NullWritable> vertex)
       throws IOException, InterruptedException {
       StringBuilder output = new StringBuilder();
       output.append(vertex.getId().get());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestAutoCheckpoint.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestAutoCheckpoint.java b/giraph-examples/src/test/java/org/apache/giraph/TestAutoCheckpoint.java
index 652913b..99ba770 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestAutoCheckpoint.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestAutoCheckpoint.java
@@ -20,9 +20,9 @@ package org.apache.giraph;
 
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.examples.SimpleCheckpointVertex;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexOutputFormat;
+import org.apache.giraph.examples.SimpleCheckpoint;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
@@ -58,15 +58,15 @@ public class TestAutoCheckpoint extends BspCase {
     }
     Path outputPath = getTempPath(getCallingMethodName());
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     conf.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
-    conf.setBoolean(SimpleCheckpointVertex.ENABLE_FAULT, true);
+    conf.setBoolean(SimpleCheckpoint.ENABLE_FAULT, true);
     conf.setInt("mapred.map.max.attempts", 4);
     // Trigger failure faster
     conf.setInt("mapred.task.timeout", 10000);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
index 38ba27f..28edbba 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
@@ -24,21 +24,22 @@ import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.examples.GeneratedVertexReader;
-import org.apache.giraph.examples.SimpleCombinerVertex;
-import org.apache.giraph.examples.SimpleFailVertex;
-import org.apache.giraph.examples.SimpleMasterComputeVertex;
-import org.apache.giraph.examples.SimpleMsgVertex;
-import org.apache.giraph.examples.SimplePageRankVertex;
-import org.apache.giraph.examples.SimplePageRankVertex.SimplePageRankVertexInputFormat;
-import org.apache.giraph.examples.SimpleShortestPathsVertex;
-import org.apache.giraph.examples.SimpleSuperstepVertex;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexOutputFormat;
+import org.apache.giraph.examples.SimpleCombinerComputation;
+import org.apache.giraph.examples.SimpleFailComputation;
+import org.apache.giraph.examples.SimpleMasterComputeComputation;
+import org.apache.giraph.examples.SimpleMsgComputation;
+import org.apache.giraph.examples.SimplePageRankComputation;
+import org.apache.giraph.examples.SimplePageRankComputation.SimplePageRankVertexInputFormat;
+import org.apache.giraph.examples.SimpleShortestPathsComputation;
+import org.apache.giraph.examples.SimpleSuperstepComputation;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexOutputFormat;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.job.HadoopUtils;
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.giraph.worker.InputSplitPathOrganizer;
 import org.apache.giraph.zk.ZooKeeperExt;
 import org.apache.hadoop.conf.Configuration;
@@ -111,13 +112,12 @@ public class
     System.out.println("testInstantiateVertex: java.class.path=" +
         System.getProperty("java.class.path"));
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleSuperstepVertex.class);
-    conf.setVertexInputFormatClass(
-        SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat.class);
+    conf.setComputationClass(SimpleSuperstepComputation.class);
+    conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf);
     ImmutableClassesGiraphConfiguration configuration =
         new ImmutableClassesGiraphConfiguration(job.getConfiguration());
-    Vertex<LongWritable, IntWritable, FloatWritable, IntWritable> vertex =
+    Vertex<LongWritable, IntWritable, FloatWritable> vertex =
         configuration.createVertex();
     vertex.initialize(new LongWritable(1), new IntWritable(1));
     System.out.println("testInstantiateVertex: Got vertex " + vertex);
@@ -133,11 +133,8 @@ public class
         byteArrayOutputStream.toString());
   }
 
-  private static class NullVertex extends Vertex<NullWritable, NullWritable,
-      NullWritable, NullWritable> {
-    @Override
-    public void compute(Iterable<NullWritable> messages) throws IOException { }
-  }
+  private static class NullComputation extends NoOpComputation<NullWritable,
+      NullWritable, NullWritable, NullWritable> { }
 
   /**
    * Test whether vertices with NullWritable for vertex value type, edge value
@@ -146,19 +143,17 @@ public class
   @Test
   public void testInstantiateNullVertex() throws IOException {
     GiraphConfiguration nullConf = new GiraphConfiguration();
-    nullConf.setVertexClass(NullVertex.class);
-    ImmutableClassesGiraphConfiguration<
-        NullWritable, NullWritable, NullWritable,
+    nullConf.setComputationClass(NullComputation.class);
+    ImmutableClassesGiraphConfiguration<NullWritable, NullWritable,
         NullWritable> immutableClassesGiraphConfiguration =
         new ImmutableClassesGiraphConfiguration<
-            NullWritable, NullWritable, NullWritable, NullWritable>(
-            nullConf);
+            NullWritable, NullWritable, NullWritable>(nullConf);
     NullWritable vertexValue =
         immutableClassesGiraphConfiguration.createVertexValue();
     NullWritable edgeValue =
         immutableClassesGiraphConfiguration.createEdgeValue();
     NullWritable messageValue =
-        immutableClassesGiraphConfiguration.createMessageValue();
+        immutableClassesGiraphConfiguration.createOutgoingMessageValue();
     assertSame(vertexValue.getClass(), NullWritable.class);
     assertSame(vertexValue, edgeValue);
     assertSame(edgeValue, messageValue);
@@ -180,7 +175,7 @@ public class
       return;
     }
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleSuperstepVertex.class);
+    conf.setComputationClass(SimpleSuperstepComputation.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf);
     conf = job.getConfiguration();
@@ -222,7 +217,7 @@ public class
     }
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleFailVertex.class);
+    conf.setComputationClass(SimpleFailComputation.class);
     conf.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf,
         getTempPath(getCallingMethodName()));
@@ -243,7 +238,7 @@ public class
     String callingMethod = getCallingMethodName();
     Path outputPath = getTempPath(callingMethod);
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleSuperstepVertex.class);
+    conf.setComputationClass(SimpleSuperstepComputation.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     GiraphJob job = prepareJob(callingMethod, conf, outputPath);
@@ -268,7 +263,7 @@ public class
   public void testBspMsg()
       throws IOException, InterruptedException, ClassNotFoundException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleMsgVertex.class);
+    conf.setComputationClass(SimpleMsgComputation.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf);
     assertTrue(job.run(true));
@@ -287,7 +282,7 @@ public class
   public void testEmptyVertexInputFormat()
       throws IOException, InterruptedException, ClassNotFoundException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleMsgVertex.class);
+    conf.setComputationClass(SimpleMsgComputation.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf);
     job.getConfiguration().setLong(GeneratedVertexReader.READER_VERTICES, 0);
@@ -305,7 +300,7 @@ public class
   public void testBspCombiner()
       throws IOException, InterruptedException, ClassNotFoundException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleCombinerVertex.class);
+    conf.setComputationClass(SimpleCombinerComputation.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setCombinerClass(SimpleSumCombiner.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf);
@@ -366,11 +361,11 @@ public class
       throws IOException, InterruptedException, ClassNotFoundException {
     Path outputPath = getTempPath(getCallingMethodName());
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleShortestPathsVertex.class);
+    conf.setComputationClass(SimpleShortestPathsComputation.class);
     conf.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
     conf.setVertexOutputFormatClass(
         JsonLongDoubleFloatDoubleVertexOutputFormat.class);
-    SimpleShortestPathsVertex.SOURCE_ID.set(conf, 0);
+    SimpleShortestPathsComputation.SOURCE_ID.set(conf, 0);
     GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
 
     assertTrue(job.run(true));
@@ -394,15 +389,15 @@ public class
     Path outputPath = getTempPath(getCallingMethodName());
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimplePageRankVertex.class);
+    conf.setComputationClass(SimplePageRankComputation.class);
     conf.setAggregatorWriterClass(TextAggregatorWriter.class);
     conf.setMasterComputeClass(
-        SimplePageRankVertex.SimplePageRankVertexMasterCompute.class);
+        SimplePageRankComputation.SimplePageRankMasterCompute.class);
     conf.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
     conf.setVertexOutputFormatClass(
-        SimplePageRankVertex.SimplePageRankVertexOutputFormat.class);
+        SimplePageRankComputation.SimplePageRankVertexOutputFormat.class);
     conf.setWorkerContextClass(
-        SimplePageRankVertex.SimplePageRankVertexWorkerContext.class);
+        SimplePageRankComputation.SimplePageRankWorkerContext.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
     GiraphConfiguration configuration = job.getConfiguration();
     Path aggregatorValues = getTempPath("aggregatorValues");
@@ -419,11 +414,11 @@ public class
     try {
       if (!runningInDistributedMode()) {
         double maxPageRank =
-            SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMax();
+            SimplePageRankComputation.SimplePageRankWorkerContext.getFinalMax();
         double minPageRank =
-            SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMin();
+            SimplePageRankComputation.SimplePageRankWorkerContext.getFinalMin();
         long numVertices =
-            SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalSum();
+            SimplePageRankComputation.SimplePageRankWorkerContext.getFinalSum();
         System.out.println("testBspPageRank: maxPageRank=" + maxPageRank +
             " minPageRank=" + minPageRank + " numVertices=" + numVertices);
 
@@ -455,7 +450,7 @@ public class
             }
           }
 
-          int maxSuperstep = SimplePageRankVertex.MAX_SUPERSTEPS;
+          int maxSuperstep = SimplePageRankComputation.MAX_SUPERSTEPS;
           assertEquals(maxSuperstep + 2, minValues.size());
           assertEquals(maxSuperstep + 2, maxValues.size());
           assertEquals(maxSuperstep + 2, vertexCounts.size());
@@ -485,17 +480,17 @@ public class
   public void testBspMasterCompute()
       throws IOException, InterruptedException, ClassNotFoundException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleMasterComputeVertex.class);
+    conf.setComputationClass(SimpleMasterComputeComputation.class);
     conf.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
     conf.setMasterComputeClass(
-        SimpleMasterComputeVertex.SimpleMasterCompute.class);
+        SimpleMasterComputeComputation.SimpleMasterCompute.class);
     conf.setWorkerContextClass(
-        SimpleMasterComputeVertex.SimpleMasterComputeWorkerContext.class);
+        SimpleMasterComputeComputation.SimpleMasterComputeWorkerContext.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf);
     assertTrue(job.run(true));
     if (!runningInDistributedMode()) {
       double finalSum =
-          SimpleMasterComputeVertex.SimpleMasterComputeWorkerContext.getFinalSum();
+          SimpleMasterComputeComputation.SimpleMasterComputeWorkerContext.getFinalSum();
       System.out.println("testBspMasterCompute: finalSum=" + finalSum);
       assertEquals(32.5, finalSum, 0d);
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestComputationState.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestComputationState.java b/giraph-examples/src/test/java/org/apache/giraph/TestComputationState.java
new file mode 100644
index 0000000..1092eac
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestComputationState.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.examples.GeneratedVertexReader;
+import org.apache.giraph.examples.SimplePageRankComputation;
+import org.apache.giraph.examples.TestComputationStateComputation;
+import org.apache.giraph.job.GiraphJob;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestComputationState extends BspCase {
+  public TestComputationState() {
+    super(TestComputationState.class.getName());
+  }
+
+  @Test
+  public void testComputationState() throws IOException,
+      ClassNotFoundException, InterruptedException {
+    if (runningInDistributedMode()) {
+      System.out.println(
+          "testComputeContext: Ignore this test in distributed mode.");
+      return;
+    }
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setComputationClass(TestComputationStateComputation.class);
+    conf.setVertexInputFormatClass(
+        SimplePageRankComputation.SimplePageRankVertexInputFormat.class);
+    conf.setWorkerContextClass(
+        TestComputationStateComputation.TestComputationStateWorkerContext.class);
+    GiraphJob job = prepareJob(getCallingMethodName(), conf);
+    // Use multithreading
+    job.getConfiguration().setNumComputeThreads(
+        TestComputationStateComputation.NUM_COMPUTE_THREADS);
+    // Increase the number of vertices
+    job.getConfiguration().setInt(
+        GeneratedVertexReader.READER_VERTICES,
+        TestComputationStateComputation.NUM_VERTICES);
+    // Increase the number of partitions
+    GiraphConstants.USER_PARTITION_COUNT.set(job.getConfiguration(),
+        TestComputationStateComputation.NUM_PARTITIONS);
+    assertTrue(job.run(true));
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestGraphPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestGraphPartitioner.java b/giraph-examples/src/test/java/org/apache/giraph/TestGraphPartitioner.java
index 12f0d8d..4537eac 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestGraphPartitioner.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestGraphPartitioner.java
@@ -21,9 +21,9 @@ package org.apache.giraph;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.examples.GeneratedVertexReader;
-import org.apache.giraph.examples.SimpleCheckpointVertex;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexOutputFormat;
+import org.apache.giraph.examples.SimpleCheckpoint;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat;
 import org.apache.giraph.integration.SuperstepHashPartitionerFactory;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.partition.HashRangePartitionerFactory;
@@ -76,12 +76,12 @@ public class TestGraphPartitioner extends BspCase {
       throws IOException, InterruptedException, ClassNotFoundException {
     Path outputPath = getTempPath("testVertexBalancer");
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     conf.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     GiraphJob job = prepareJob("testVertexBalancer", conf, outputPath);
@@ -94,12 +94,12 @@ public class TestGraphPartitioner extends BspCase {
     FileSystem hdfs = FileSystem.get(job.getConfiguration());
 
     conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     conf.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     outputPath = getTempPath("testHashPartitioner");
@@ -109,12 +109,12 @@ public class TestGraphPartitioner extends BspCase {
 
     outputPath = getTempPath("testSuperstepHashPartitioner");
     conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     conf.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     job = prepareJob("testSuperstepHashPartitioner", conf, outputPath);
@@ -127,12 +127,12 @@ public class TestGraphPartitioner extends BspCase {
 
     job = new GiraphJob("testHashRangePartitioner");
     setupConfiguration(job);
-    job.getConfiguration().setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    job.getConfiguration().setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     job.getConfiguration().setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     job.getConfiguration().setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     job.getConfiguration().setVertexInputFormatClass(
         SimpleSuperstepVertexInputFormat.class);
     job.getConfiguration().setVertexOutputFormatClass(
@@ -146,12 +146,12 @@ public class TestGraphPartitioner extends BspCase {
 
     outputPath = getTempPath("testReverseIdSuperstepHashPartitioner");
     conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     conf.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     job = prepareJob("testReverseIdSuperstepHashPartitioner", conf,
@@ -165,12 +165,12 @@ public class TestGraphPartitioner extends BspCase {
 
     job = new GiraphJob("testSimpleRangePartitioner");
     setupConfiguration(job);
-    job.getConfiguration().setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    job.getConfiguration().setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     job.getConfiguration().setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     job.getConfiguration().setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     job.getConfiguration().setVertexInputFormatClass(
         SimpleSuperstepVertexInputFormat.class);
     job.getConfiguration().setVertexOutputFormatClass(

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestManualCheckpoint.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestManualCheckpoint.java b/giraph-examples/src/test/java/org/apache/giraph/TestManualCheckpoint.java
index 766e1af..fc94480 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestManualCheckpoint.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestManualCheckpoint.java
@@ -20,9 +20,9 @@ package org.apache.giraph;
 
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.examples.SimpleCheckpointVertex;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexOutputFormat;
+import org.apache.giraph.examples.SimpleCheckpoint;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -54,12 +54,12 @@ public class TestManualCheckpoint extends BspCase {
     Path checkpointsDir = getTempPath("checkPointsForTesting");
     Path outputPath = getTempPath(getCallingMethodName());
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     conf.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
@@ -75,7 +75,7 @@ public class TestManualCheckpoint extends BspCase {
     if (!runningInDistributedMode()) {
       FileStatus fileStatus = getSinglePartFileStatus(job.getConfiguration(),
           outputPath);
-      idSum = SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext
+      idSum = SimpleCheckpoint.SimpleCheckpointVertexWorkerContext
           .getFinalSum();
       System.out.println("testBspCheckpoint: idSum = " + idSum +
           " fileLen = " + fileStatus.getLen());
@@ -86,25 +86,25 @@ public class TestManualCheckpoint extends BspCase {
         " with checkpoint path = " + checkpointsDir);
     outputPath = getTempPath(getCallingMethodName() + "Restarted");
     conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setWorkerContextClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext.class);
+        SimpleCheckpoint.SimpleCheckpointVertexWorkerContext.class);
     conf.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     GiraphJob restartedJob = prepareJob(getCallingMethodName() + "Restarted",
         conf, outputPath);
     configuration.setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     GiraphConstants.CHECKPOINT_DIRECTORY.set(restartedJob.getConfiguration(),
         checkpointsDir.toString());
 
     assertTrue(restartedJob.run(true));
     if (!runningInDistributedMode()) {
       long idSumRestarted =
-          SimpleCheckpointVertex.SimpleCheckpointVertexWorkerContext
+          SimpleCheckpoint.SimpleCheckpointVertexWorkerContext
               .getFinalSum();
       System.out.println("testBspCheckpoint: idSumRestarted = " +
           idSumRestarted);


[06/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/comm/TestMessageStores.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/TestMessageStores.java b/giraph-core/src/test/java/org/apache/giraph/comm/TestMessageStores.java
index 53b8a24..4e8041a 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/TestMessageStores.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/TestMessageStores.java
@@ -37,8 +37,8 @@ import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
 import org.apache.giraph.utils.CollectionUtils;
+import org.apache.giraph.utils.IntNoOpComputation;
 import org.apache.giraph.utils.MockUtils;
-import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
@@ -68,35 +68,28 @@ import static org.junit.Assert.assertTrue;
 /** Test for different types of message stores */
 public class TestMessageStores {
   private static File directory;
-  private static ImmutableClassesGiraphConfiguration config;
+  private static ImmutableClassesGiraphConfiguration<IntWritable,
+      IntWritable, IntWritable> config;
   private static TestData testData;
   private static
-  CentralizedServiceWorker<IntWritable, IntWritable, IntWritable, IntWritable>
-      service;
+  CentralizedServiceWorker<IntWritable, IntWritable, IntWritable> service;
   /**
    * Pseudo-random number generator with the same seed to help with
    * debugging)
    */
   private static final Random RANDOM = new Random(101);
 
-  private static class IntVertex extends Vertex<IntWritable,
-        IntWritable, IntWritable, IntWritable> {
-
-    @Override
-    public void compute(Iterable<IntWritable> messages) throws IOException {
-    }
-  }
-
   @Before
   public void prepare() throws IOException {
     directory = Files.createTempDir();
 
     Configuration.addDefaultResource("giraph-site.xml");
     GiraphConfiguration initConfig = new GiraphConfiguration();
-    initConfig.setVertexClass(IntVertex.class);
+    initConfig.setComputationClass(IntNoOpComputation.class);
     GiraphConstants.MESSAGES_DIRECTORY.set(
         initConfig, new File(directory, "giraph_messages").toString());
-    config = new ImmutableClassesGiraphConfiguration(initConfig);
+    config = new ImmutableClassesGiraphConfiguration<IntWritable,
+        IntWritable, IntWritable>(initConfig);
 
     testData = new TestData();
     testData.maxId = 1000000;
@@ -158,18 +151,18 @@ public class TestMessageStores {
      * @param config  Hadoop configuration
      */
     InputMessageStore(
-        CentralizedServiceWorker<IntWritable, ?, ?, IntWritable> service,
-        ImmutableClassesGiraphConfiguration<IntWritable, ?, ?,
-            IntWritable> config,
+        CentralizedServiceWorker<IntWritable, ?, ?> service,
+        ImmutableClassesGiraphConfiguration<IntWritable, ?, ?> config,
         Map<IntWritable, Collection<IntWritable>> inputMap) throws IOException {
-      super(service, config);
+      super(IntWritable.class, service, config);
       // Adds all the messages to the store
       for (Map.Entry<IntWritable, Collection<IntWritable>> entry :
           inputMap.entrySet()) {
         int partitionId = getPartitionId(entry.getKey());
         ByteArrayVertexIdMessages<IntWritable, IntWritable>
             byteArrayVertexIdMessages =
-            new ByteArrayVertexIdMessages<IntWritable, IntWritable>();
+            new ByteArrayVertexIdMessages<IntWritable,
+                IntWritable>(IntWritable.class);
         byteArrayVertexIdMessages.setConf(config);
         byteArrayVertexIdMessages.initialize();
         for (IntWritable message : entry.getValue()) {
@@ -238,7 +231,7 @@ public class TestMessageStores {
     messageStore.write(out);
     out.close();
 
-    messageStore = messageStoreFactory.newStore();
+    messageStore = messageStoreFactory.newStore(IntWritable.class);
 
     DataInputStream in = new DataInputStream(new BufferedInputStream(
         (new FileInputStream(file))));
@@ -255,7 +248,7 @@ public class TestMessageStores {
       TestData testData) throws IOException {
     SortedMap<IntWritable, Collection<IntWritable>> messages =
         new TreeMap<IntWritable, Collection<IntWritable>>();
-    S messageStore = messageStoreFactory.newStore();
+    S messageStore = messageStoreFactory.newStore(IntWritable.class);
     putNTimes(messageStore, messages, testData);
     assertTrue(equalMessages(messageStore, messages));
     messageStore.clearAll();
@@ -268,7 +261,8 @@ public class TestMessageStores {
   public void testByteArrayMessagesPerVertexStore() {
     try {
       testMessageStore(
-          ByteArrayMessagesPerVertexStore.newFactory(service, config),
+          ByteArrayMessagesPerVertexStore.<IntWritable, IntWritable>newFactory(
+              service, config),
           testData);
     } catch (IOException e) {
       e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/conf/TestObjectCreation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/conf/TestObjectCreation.java b/giraph-core/src/test/java/org/apache/giraph/conf/TestObjectCreation.java
index 9075145..4e38bff 100644
--- a/giraph-core/src/test/java/org/apache/giraph/conf/TestObjectCreation.java
+++ b/giraph-core/src/test/java/org/apache/giraph/conf/TestObjectCreation.java
@@ -18,10 +18,10 @@
 
 package org.apache.giraph.conf;
 
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.time.SystemTime;
 import org.apache.giraph.time.Time;
 import org.apache.giraph.time.Times;
+import org.apache.giraph.utils.LongNoOpComputation;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -51,7 +51,7 @@ public class TestObjectCreation {
   private long total = 0;
   private long expected = COUNT * (COUNT - 1) / 2L;
   private ImmutableClassesGiraphConfiguration<LongWritable, LongWritable,
-      LongWritable, LongWritable> configuration;
+      LongWritable> configuration;
 
   @Before
   public void setUp() {
@@ -59,11 +59,12 @@ public class TestObjectCreation {
     GiraphConstants.VERTEX_ID_CLASS.set(conf, IntWritable.class);
     GiraphConstants.VERTEX_VALUE_CLASS.set(conf, LongWritable.class);
     GiraphConstants.EDGE_VALUE_CLASS.set(conf, DoubleWritable.class);
-    GiraphConstants.MESSAGE_VALUE_CLASS.set(conf, LongWritable.class);
-    conf.setVertexClass(ImmutableVertex.class);
+    GiraphConstants.INCOMING_MESSAGE_VALUE_CLASS.set(conf, LongWritable.class);
+    GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS.set(conf, LongWritable.class);
+    conf.setComputationClass(LongNoOpComputation.class);
     configuration =
         new ImmutableClassesGiraphConfiguration<LongWritable, LongWritable,
-            LongWritable, LongWritable>(conf);
+            LongWritable>(conf);
     total = 0;
     System.gc();
   }
@@ -161,15 +162,8 @@ public class TestObjectCreation {
     }
   }
 
-  private static class ImmutableVertex extends Vertex<LongWritable,
-        LongWritable, LongWritable, LongWritable> {
-    @Override
-    public void compute(Iterable<LongWritable> messages) throws IOException {
-    }
-  }
-
   private ImmutableClassesGiraphConfiguration<LongWritable, LongWritable,
-      LongWritable, LongWritable> getConfiguration() {
+      LongWritable> getConfiguration() {
     return configuration;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
index 15d2bb0..b6e17fd 100644
--- a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
@@ -18,6 +18,7 @@
 package org.apache.giraph.graph;
 
 import com.google.common.collect.Lists;
+
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.ArrayListEdges;
@@ -36,6 +37,7 @@ import org.apache.giraph.time.Times;
 import org.apache.giraph.utils.DynamicChannelBufferInputStream;
 import org.apache.giraph.utils.DynamicChannelBufferOutputStream;
 import org.apache.giraph.utils.EdgeIterables;
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.giraph.utils.UnsafeByteArrayInputStream;
 import org.apache.giraph.utils.UnsafeByteArrayOutputStream;
 import org.apache.giraph.utils.WritableUtils;
@@ -70,11 +72,8 @@ public class TestVertexAndEdges {
   /**
    * Dummy concrete vertex.
    */
-  public static class TestVertex extends Vertex<LongWritable, FloatWritable,
-        DoubleWritable, LongWritable> {
-    @Override
-    public void compute(Iterable<LongWritable> messages) { }
-  }
+  public static class TestComputation extends NoOpComputation<LongWritable,
+      FloatWritable, DoubleWritable, LongWritable> { }
 
   /**
    * A basic {@link org.apache.giraph.edge.OutEdges} implementation that doesn't provide any
@@ -160,10 +159,10 @@ public class TestVertexAndEdges {
     edgesClasses.add(LongDoubleHashMapEdges.class);
   }
 
-  private Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
+  private Vertex<LongWritable, FloatWritable, DoubleWritable>
   instantiateVertex(Class<? extends OutEdges> edgesClass) {
     GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
-    giraphConfiguration.setVertexClass(TestVertex.class);
+    giraphConfiguration.setComputationClass(TestComputation.class);
     giraphConfiguration.setOutEdgesClass(edgesClass);
     ImmutableClassesGiraphConfiguration immutableClassesGiraphConfiguration =
         new ImmutableClassesGiraphConfiguration(giraphConfiguration);
@@ -175,7 +174,7 @@ public class TestVertexAndEdges {
    */
   @Test
   public void testVertexIdAndValue() {
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
+    Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         instantiateVertex(ArrayListEdges.class);
     assertNotNull(vertex);
     vertex.initialize(new LongWritable(7), new FloatWritable(3.0f));
@@ -189,7 +188,7 @@ public class TestVertexAndEdges {
   instantiateOutEdges(Class<? extends OutEdges> edgesClass) {
     GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
     // Needed to extract type arguments in ReflectionUtils.
-    giraphConfiguration.setVertexClass(TestVertex.class);
+    giraphConfiguration.setComputationClass(TestComputation.class);
     giraphConfiguration.setOutEdgesClass(edgesClass);
     ImmutableClassesGiraphConfiguration immutableClassesGiraphConfiguration =
         new ImmutableClassesGiraphConfiguration(giraphConfiguration);
@@ -208,7 +207,7 @@ public class TestVertexAndEdges {
   }
 
   private void testEdgesClass(Class<? extends OutEdges> edgesClass) {
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
+    Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         instantiateVertex(edgesClass);
     OutEdges<LongWritable, DoubleWritable> outEdges =
         instantiateOutEdges(edgesClass);
@@ -252,7 +251,7 @@ public class TestVertexAndEdges {
   }
 
   private void testMutateEdgesClass(Class<? extends OutEdges> edgesClass) {
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
+    Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         instantiateVertex(edgesClass);
     OutEdges<LongWritable, DoubleWritable> outEdges =
         instantiateOutEdges(edgesClass);
@@ -339,9 +338,9 @@ public class TestVertexAndEdges {
     }
   }
 
-  private Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
+  private Vertex<LongWritable, FloatWritable, DoubleWritable>
   buildVertex(Class<? extends OutEdges> edgesClass) {
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
+    Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         instantiateVertex(edgesClass);
     OutEdges<LongWritable, DoubleWritable> outEdges =
         instantiateOutEdges(edgesClass);
@@ -362,7 +361,7 @@ public class TestVertexAndEdges {
 
   private void testSerializeOutEdgesClass(
       Class<? extends OutEdges> edgesClass) {
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
+    Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         buildVertex(edgesClass);
 
     long serializeNanosStart;
@@ -381,7 +380,7 @@ public class TestVertexAndEdges {
         (byteArray.length * 1f * Time.NS_PER_SECOND / serializeNanos) +
         " bytes / sec for " + edgesClass.getName());
 
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
+    Vertex<LongWritable, FloatWritable, DoubleWritable>
         readVertex = buildVertex(edgesClass);
     
     long deserializeNanosStart;
@@ -407,7 +406,7 @@ public class TestVertexAndEdges {
   private void testDynamicChannelBufferSerializeOutEdgesClass(
       Class<? extends OutEdges> edgesClass)
       throws IOException {
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
+    Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         buildVertex(edgesClass);
 
     long serializeNanosStart;
@@ -430,7 +429,7 @@ public class TestVertexAndEdges {
             Time.NS_PER_SECOND / serializeNanos) +
         " bytes / sec for " + edgesClass.getName());
 
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
+    Vertex<LongWritable, FloatWritable, DoubleWritable>
         readVertex = buildVertex(edgesClass);
 
     long deserializeNanosStart;
@@ -462,7 +461,7 @@ public class TestVertexAndEdges {
   private void testUnsafeSerializeOutEdgesClass(
       Class<? extends OutEdges> edgesClass)
       throws IOException {
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
+    Vertex<LongWritable, FloatWritable, DoubleWritable> vertex =
         buildVertex(edgesClass);
 
     long serializeNanosStart;
@@ -487,7 +486,7 @@ public class TestVertexAndEdges {
             Time.NS_PER_SECOND / serializeNanos) +
         " bytes / sec for " + edgesClass.getName());
 
-    Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
+    Vertex<LongWritable, FloatWritable, DoubleWritable>
         readVertex = buildVertex(edgesClass);
 
     long deserializeNanosStart;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
index 6849e3a..fdfb5e9 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
@@ -22,15 +22,16 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.formats.AdjacencyListTextVertexOutputFormat;
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.junit.Before;
@@ -41,25 +42,20 @@ import static org.mockito.Mockito.*;
 public class TestAdjacencyListTextVertexOutputFormat extends AdjacencyListTextVertexOutputFormat<Text, DoubleWritable, DoubleWritable> {
   /** Test configuration */
   private ImmutableClassesGiraphConfiguration<Text,
-      DoubleWritable, DoubleWritable, Writable> conf;
+      DoubleWritable, DoubleWritable> conf;
 
   /**
    * Dummy class to allow ImmutableClassesGiraphConfiguration to be created.
    */
-  public static class DummyVertex extends Vertex<Text, DoubleWritable,
-      DoubleWritable, DoubleWritable> {
-    @Override
-    public void compute(Iterable<DoubleWritable> messages) throws IOException {
-      // Do nothing
-    }
-  }
+  public static class DummyComputation extends NoOpComputation<Text,
+      DoubleWritable, DoubleWritable, DoubleWritable> { }
 
   @Before
   public void setUp() {
     GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
-    giraphConfiguration.setVertexClass(DummyVertex.class);
+    giraphConfiguration.setComputationClass(DummyComputation.class);
     conf = new ImmutableClassesGiraphConfiguration<Text,
-        DoubleWritable, DoubleWritable, Writable>(giraphConfiguration);
+        DoubleWritable, DoubleWritable>(giraphConfiguration);
   }
 
   protected AdjacencyListTextVertexWriter createVertexWriter(
@@ -104,7 +100,6 @@ public class TestAdjacencyListTextVertexOutputFormat extends AdjacencyListTextVe
     Vertex vertex = mock(Vertex.class);
     when(vertex.getId()).thenReturn(new Text("San Francisco"));
     when(vertex.getValue()).thenReturn(new DoubleWritable(0d));
-    when(vertex.getTotalNumEdges()).thenReturn(2l);
     List<Edge<Text, DoubleWritable>> cities = Lists.newArrayList();
     Collections.addAll(cities,
         EdgeFactory.create(new Text("Los Angeles"), new DoubleWritable(347.16)),
@@ -133,7 +128,6 @@ public class TestAdjacencyListTextVertexOutputFormat extends AdjacencyListTextVe
     Vertex vertex = mock(Vertex.class);
     when(vertex.getId()).thenReturn(new Text("San Francisco"));
     when(vertex.getValue()).thenReturn(new DoubleWritable(0d));
-    when(vertex.getTotalNumEdges()).thenReturn(2l);
     List<Edge<Text, DoubleWritable>> cities = Lists.newArrayList();
     Collections.addAll(cities,
         EdgeFactory.create(new Text("Los Angeles"), new DoubleWritable(347.16)),

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
index cb1a8da..327aaa3 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
@@ -23,14 +23,15 @@ import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
 import org.apache.giraph.io.formats.IntIntTextVertexValueInputFormat;
 import org.apache.giraph.io.formats.IntNullReverseTextEdgeInputFormat;
 import org.apache.giraph.io.formats.IntNullTextEdgeInputFormat;
+import org.apache.giraph.utils.ComputationCountEdges;
+import org.apache.giraph.utils.IntIntNullNoOpComputation;
 import org.apache.giraph.utils.InternalVertexRunner;
-import org.apache.giraph.vertices.IntIntNullVertexDoNothing;
-import org.apache.giraph.vertices.VertexCountEdges;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.junit.Test;
@@ -65,7 +66,7 @@ public class TestEdgeInput extends BspCase {
     };
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
@@ -94,7 +95,7 @@ public class TestEdgeInput extends BspCase {
     };
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setEdgeInputFormatClass(IntNullReverseTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
@@ -130,7 +131,7 @@ public class TestEdgeInput extends BspCase {
     };
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(IntIntNullVertexDoNothing.class);
+    conf.setComputationClass(IntIntNullNoOpComputation.class);
     conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(IntIntTextVertexValueInputFormat.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
@@ -161,7 +162,7 @@ public class TestEdgeInput extends BspCase {
     assertEquals(3, (int) values.get(5));
 
     conf = new GiraphConfiguration();
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(IntIntTextVertexValueInputFormat.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
@@ -191,7 +192,7 @@ public class TestEdgeInput extends BspCase {
     };
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(TestVertexCheckEdgesType.class);
+    conf.setComputationClass(TestComputationCheckEdgesType.class);
     conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setInputOutEdgesClass(TestOutEdgesFilterEven.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
@@ -210,12 +211,14 @@ public class TestEdgeInput extends BspCase {
     assertEquals(0, (int) values.get(4));
   }
 
-  public static class TestVertexCheckEdgesType extends VertexCountEdges {
+  public static class TestComputationCheckEdgesType extends
+      ComputationCountEdges {
     @Override
-    public void compute(Iterable<NullWritable> messages) throws IOException {
-      assertFalse(getEdges() instanceof TestOutEdgesFilterEven);
-      assertTrue(getEdges() instanceof ByteArrayEdges);
-      super.compute(messages);
+    public void compute(Vertex<IntWritable, IntWritable, NullWritable> vertex,
+        Iterable<NullWritable> messages) throws IOException {
+      assertFalse(vertex.getEdges() instanceof TestOutEdgesFilterEven);
+      assertTrue(vertex.getEdges() instanceof ByteArrayEdges);
+      super.compute(vertex, messages);
     }
   }
 
@@ -223,7 +226,7 @@ public class TestEdgeInput extends BspCase {
       implements VertexValueFactory<IntWritable> {
     @Override
     public void initialize(ImmutableClassesGiraphConfiguration<?, IntWritable,
-            ?, ?> configuration) { }
+        ?> configuration) { }
 
     @Override
     public IntWritable createVertexValue() {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/io/TestFilters.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestFilters.java b/giraph-core/src/test/java/org/apache/giraph/io/TestFilters.java
index 83a366d..86c37ea 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestFilters.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestFilters.java
@@ -26,9 +26,9 @@ import org.apache.giraph.io.filters.VertexInputFilter;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
 import org.apache.giraph.io.formats.IntIntTextVertexValueInputFormat;
 import org.apache.giraph.io.formats.IntNullTextEdgeInputFormat;
+import org.apache.giraph.utils.ComputationCountEdges;
+import org.apache.giraph.utils.IntNoOpComputation;
 import org.apache.giraph.utils.InternalVertexRunner;
-import org.apache.giraph.vertices.IntIntNullVertexDoNothing;
-import org.apache.giraph.vertices.VertexCountEdges;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.junit.Test;
@@ -60,7 +60,7 @@ public class TestFilters extends BspCase {
     };
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(VertexCountEdges.class);
+    conf.setComputationClass(ComputationCountEdges.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
     conf.setEdgeInputFilterClass(EdgeFilter.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
@@ -74,9 +74,9 @@ public class TestFilters extends BspCase {
   }
 
   public static class VertexFilter implements VertexInputFilter<IntWritable,
-      NullWritable, NullWritable, NullWritable> {
+      NullWritable, NullWritable> {
     @Override
-    public boolean dropVertex(Vertex<IntWritable, NullWritable, NullWritable,
+    public boolean dropVertex(Vertex<IntWritable, NullWritable,
         NullWritable> vertex) {
       int id = vertex.getId().get();
       return id == 2 || id == 3;
@@ -93,7 +93,7 @@ public class TestFilters extends BspCase {
     };
 
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(IntIntNullVertexDoNothing.class);
+    conf.setComputationClass(IntNoOpComputation.class);
     conf.setVertexInputFormatClass(IntIntTextVertexValueInputFormat.class);
     conf.setVertexInputFilterClass(VertexFilter.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java
index ff4d2b1..68059f9 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestIdWithValueTextOutputFormat.java
@@ -20,10 +20,12 @@ package org.apache.giraph.io;
 
 import java.io.IOException;
 import java.util.ArrayList;
+
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -38,24 +40,19 @@ public class TestIdWithValueTextOutputFormat extends
     IdWithValueTextOutputFormat<Text, DoubleWritable, Writable> {
   /** Test configuration */
   private ImmutableClassesGiraphConfiguration<
-      Text, DoubleWritable, Writable, Writable> conf;
+      Text, DoubleWritable, Writable> conf;
   /**
    * Dummy class to allow ImmutableClassesGiraphConfiguration to be created.
    */
-  public static class DummyVertex extends Vertex<Text, DoubleWritable,
-      DoubleWritable, DoubleWritable> {
-    @Override
-    public void compute(Iterable<DoubleWritable> messages) throws IOException {
-      // Do nothing
-    }
-  }
+  public static class DummyComputation extends NoOpComputation<Text,
+      DoubleWritable, DoubleWritable, DoubleWritable> { }
 
   @Before
   public void setUp() {
     GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
-    giraphConfiguration.setVertexClass(DummyVertex.class);
+    giraphConfiguration.setComputationClass(DummyComputation.class);
     conf = new ImmutableClassesGiraphConfiguration<Text, DoubleWritable,
-        Writable, Writable>(giraphConfiguration);
+        Writable>(giraphConfiguration);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/io/TestJsonBase64Format.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestJsonBase64Format.java b/giraph-core/src/test/java/org/apache/giraph/io/TestJsonBase64Format.java
index a6aa78a..3b0dc63 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestJsonBase64Format.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestJsonBase64Format.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.io;
 
 import org.apache.giraph.BspCase;
-import org.apache.giraph.benchmark.WeightedPageRankVertex;
+import org.apache.giraph.benchmark.WeightedPageRankComputation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.io.formats.GiraphFileInputFormat;
 import org.apache.giraph.io.formats.JsonBase64VertexInputFormat;
@@ -61,7 +61,7 @@ public class TestJsonBase64Format extends BspCase {
 
     Path outputPath = getTempPath(getCallingMethodName());
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(WeightedPageRankVertex.class);
+    conf.setComputationClass(WeightedPageRankComputation.class);
     conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);
     conf.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
@@ -69,31 +69,34 @@ public class TestJsonBase64Format extends BspCase {
         PseudoRandomInputFormatConstants.AGGREGATE_VERTICES, 101);
     job.getConfiguration().setLong(
         PseudoRandomInputFormatConstants.EDGES_PER_VERTEX, 2);
-    job.getConfiguration().setInt(WeightedPageRankVertex.SUPERSTEP_COUNT, 2);
+    job.getConfiguration().setInt(
+        WeightedPageRankComputation.SUPERSTEP_COUNT, 2);
 
     assertTrue(job.run(true));
 
     Path outputPath2 = getTempPath(getCallingMethodName() + "2");
     conf = new GiraphConfiguration();
-    conf.setVertexClass(WeightedPageRankVertex.class);
+    conf.setComputationClass(WeightedPageRankComputation.class);
     conf.setVertexInputFormatClass(JsonBase64VertexInputFormat.class);
     conf.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class);
     job = prepareJob(getCallingMethodName(), conf, outputPath2);
-    job.getConfiguration().setInt(WeightedPageRankVertex.SUPERSTEP_COUNT, 3);
+    job.getConfiguration().setInt(
+        WeightedPageRankComputation.SUPERSTEP_COUNT, 3);
     GiraphFileInputFormat.addVertexInputPath(
       job.getInternalJob().getConfiguration(), outputPath);
     assertTrue(job.run(true));
 
     Path outputPath3 = getTempPath(getCallingMethodName() + "3");
     conf = new GiraphConfiguration();
-    conf.setVertexClass(WeightedPageRankVertex.class);
+    conf.setComputationClass(WeightedPageRankComputation.class);
     conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);
     conf.setVertexOutputFormatClass(JsonBase64VertexOutputFormat.class);
     job = prepareJob(getCallingMethodName(), conf, outputPath3);
     conf = job.getConfiguration();
     conf.setLong(PseudoRandomInputFormatConstants.AGGREGATE_VERTICES, 101);
     conf.setLong(PseudoRandomInputFormatConstants.EDGES_PER_VERTEX, 2);
-    conf.setInt(WeightedPageRankVertex.SUPERSTEP_COUNT, 5);
+    conf.setInt(
+        WeightedPageRankComputation.SUPERSTEP_COUNT, 5);
     assertTrue(job.run(true));
 
     assertEquals(101, getNumResults(conf, outputPath));

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
index 601c824..c67a1c4 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
@@ -19,18 +19,19 @@ package org.apache.giraph.io;
 
 
 import java.io.IOException;
+
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.formats.AdjacencyListTextVertexInputFormat;
 import org.apache.giraph.io.formats.LongDoubleDoubleAdjacencyListVertexInputFormat;
+
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
@@ -38,7 +39,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import static org.apache.giraph.io.TestTextDoubleDoubleAdjacencyListVertexInputFormat.assertValidVertex;
-import static org.apache.giraph.io.TestTextDoubleDoubleAdjacencyListVertexInputFormat.setGraphState;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -47,19 +47,17 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
 
   private RecordReader<LongWritable, Text> rr;
   private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, Writable> conf;
+      DoubleWritable> conf;
   private TaskAttemptContext tac;
-  private GraphState<LongWritable, DoubleWritable, DoubleWritable, BooleanWritable> graphState;
 
   @Before
   public void setUp() throws IOException, InterruptedException {
     rr = mock(RecordReader.class);
     when(rr.nextKeyValue()).thenReturn(true);
     GiraphConfiguration giraphConf = new GiraphConfiguration();
-    giraphConf.setVertexClass(DummyVertex.class);
+    giraphConf.setComputationClass(DummyComputation.class);
     conf = new ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-        DoubleWritable, Writable>(giraphConf);
-    graphState = mock(GraphState.class);
+        DoubleWritable>(giraphConf);
     tac = mock(TaskAttemptContext.class);
     when(tac.getConfiguration()).thenReturn(conf);
   }
@@ -129,10 +127,9 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
     vr.initialize(null, tac);
 
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+    Vertex<LongWritable, DoubleWritable, DoubleWritable>
         vertex = vr.getCurrentVertex();
-    setGraphState(vertex, graphState);
-    assertValidVertex(conf, graphState, vertex,
+    assertValidVertex(conf, vertex,
         new LongWritable(42), new DoubleWritable(0.1),
         EdgeFactory.create(new LongWritable(99), new DoubleWritable(0.2)),
         EdgeFactory.create(new LongWritable(2000), new DoubleWritable(0.3)),
@@ -150,20 +147,14 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
     vr.setConf(conf);
     vr.initialize(null, tac);
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+    Vertex<LongWritable, DoubleWritable, DoubleWritable>
         vertex = vr.getCurrentVertex();
-    setGraphState(vertex, graphState);
-    assertValidVertex(conf, graphState, vertex,
+    assertValidVertex(conf, vertex,
         new LongWritable(12345), new DoubleWritable(42.42),
        EdgeFactory.create(new LongWritable(9999999), new DoubleWritable(99.9)));
     assertEquals(vertex.getNumEdges(), 1);
   }
 
-  public static class DummyVertex extends Vertex<LongWritable, DoubleWritable,
-      DoubleWritable, BooleanWritable> {
-    @Override
-    public void compute(Iterable<BooleanWritable> messages) throws IOException {
-      // ignore
-    }
-  }
+  public static class DummyComputation extends NoOpComputation<LongWritable,
+      DoubleWritable, DoubleWritable, BooleanWritable> { }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
index 4ee8ea1..8034052 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
@@ -19,13 +19,13 @@ package org.apache.giraph.io;
 
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.io.formats.AdjacencyListTextVertexInputFormat;
 import org.apache.giraph.io.formats.TextDoubleDoubleAdjacencyListVertexInputFormat;
 import org.apache.giraph.utils.EdgeIterables;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -39,7 +39,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.lang.reflect.Method;
 import java.util.Arrays;
 
 import static org.junit.Assert.assertEquals;
@@ -48,23 +47,21 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoubleDoubleAdjacencyListVertexInputFormat<BooleanWritable> {
+public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoubleDoubleAdjacencyListVertexInputFormat {
 
   private RecordReader<LongWritable, Text> rr;
   private ImmutableClassesGiraphConfiguration<Text, DoubleWritable,
-      DoubleWritable, Writable> conf;
+      DoubleWritable> conf;
   private TaskAttemptContext tac;
-  private GraphState<Text, DoubleWritable, DoubleWritable, BooleanWritable> graphState;
 
   @Before
   public void setUp() throws IOException, InterruptedException {
     rr = mock(RecordReader.class);
     when(rr.nextKeyValue()).thenReturn(true).thenReturn(false);
     GiraphConfiguration giraphConf = new GiraphConfiguration();
-    giraphConf.setVertexClass(DummyVertex.class);
+    giraphConf.setComputationClass(DummyComputation.class);
     conf = new ImmutableClassesGiraphConfiguration<Text, DoubleWritable,
-        DoubleWritable, Writable>(giraphConf);
-    graphState = mock(GraphState.class);
+        DoubleWritable>(giraphConf);
     tac = mock(TaskAttemptContext.class);
     when(tac.getConfiguration()).thenReturn(conf);
   }
@@ -121,33 +118,23 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
     }
   }
 
-  public static void setGraphState(Vertex vertex, GraphState graphState) throws Exception {
-    Class<? extends Vertex> c = Vertex.class;
-    Method m = c.getDeclaredMethod("setGraphState", GraphState.class);
-    m.setAccessible(true);
-    m.invoke(vertex, graphState);
-  }
-
   public static <I extends WritableComparable, V extends Writable,
       E extends WritableComparable> void assertValidVertex(
-      ImmutableClassesGiraphConfiguration<I, V, E, ?> conf,
-      GraphState<I, V, E, ?> graphState,
-      Vertex<I, V, E, ?> actual,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf,
+      Vertex<I, V, E> actual,
       I expectedId,
       V expectedValue,
       Edge<I, E>... edges) throws Exception {
-    Vertex<I, V, E, ?> expected = conf.createVertex();
-    setGraphState(expected, graphState);
+    Vertex<I, V, E> expected = conf.createVertex();
     expected.initialize(expectedId, expectedValue, Arrays.asList(edges));
     assertValid(expected, actual);
   }
 
   public static <I extends WritableComparable, V extends Writable,
       E extends WritableComparable> void assertValid(
-      Vertex<I, V, E, ?> expected, Vertex<I, V, E, ?> actual) {
+      Vertex<I, V, E> expected, Vertex<I, V, E> actual) {
     assertEquals(expected.getId(), actual.getId());
     assertEquals(expected.getValue(), actual.getValue());
-    assertEquals(expected.getTotalNumEdges(), actual.getTotalNumEdges());
     assertTrue(EdgeIterables.equals(expected.getEdges(), actual.getEdges()));
   }
 
@@ -160,9 +147,8 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
     vr.setConf(conf);
     vr.initialize(null, tac);
     assertTrue("Should have been able to add a vertex", vr.nextVertex());
-    Vertex<Text, DoubleWritable, DoubleWritable, ?> vertex = vr.getCurrentVertex();
-    setGraphState(vertex, graphState);
-    assertValidVertex(conf, graphState, vertex,
+    Vertex<Text, DoubleWritable, DoubleWritable> vertex = vr.getCurrentVertex();
+    assertValidVertex(conf, vertex,
         new Text("Hi"), new DoubleWritable(0),
         EdgeFactory.create(new Text("Ciao"), new DoubleWritable(1.123d)),
         EdgeFactory.create(new Text("Bomdia"), new DoubleWritable(2.234d)),
@@ -187,9 +173,8 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
     vr.setConf(conf);
     vr.initialize(null, tac);
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<Text, DoubleWritable, DoubleWritable, ?> vertex = vr.getCurrentVertex();
-    setGraphState(vertex, graphState);
-    assertValidVertex(conf, graphState, vertex,
+    Vertex<Text, DoubleWritable, DoubleWritable> vertex = vr.getCurrentVertex();
+    assertValidVertex(conf, vertex,
         new Text("BYE"), new DoubleWritable(0.01d),
         EdgeFactory.create(new Text("CIAO"), new DoubleWritable(1.001d)),
         EdgeFactory.create(new Text("TCHAU"), new DoubleWritable(2.0001d)),
@@ -208,19 +193,13 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
     vr.setConf(conf);
     vr.initialize(null, tac);
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<Text, DoubleWritable, DoubleWritable, ?> vertex = vr.getCurrentVertex();
-    setGraphState(vertex, graphState);
-    assertValidVertex(conf, graphState, vertex,
+    Vertex<Text, DoubleWritable, DoubleWritable> vertex = vr.getCurrentVertex();
+    assertValidVertex(conf, vertex,
         new Text("alpha"), new DoubleWritable(42d),
         EdgeFactory.create(new Text("beta"), new DoubleWritable(99d)));
     assertEquals(vertex.getNumEdges(), 1);
   }
 
-  public static class DummyVertex extends Vertex<Text, DoubleWritable,
-            DoubleWritable, BooleanWritable> {
-    @Override
-    public void compute(Iterable<BooleanWritable> messages) throws IOException {
-      // ignore
-    }
-  }
+  public static class DummyComputation extends NoOpComputation<Text,
+      DoubleWritable, DoubleWritable, BooleanWritable> { }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/master/TestComputationCombinerTypes.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/master/TestComputationCombinerTypes.java b/giraph-core/src/test/java/org/apache/giraph/master/TestComputationCombinerTypes.java
new file mode 100644
index 0000000..8ae09bc
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/master/TestComputationCombinerTypes.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.master;
+
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.graph.Computation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.IntNoOpComputation;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/** Test type verification when switching computation and combiner types */
+public class TestComputationCombinerTypes {
+  @Test
+  public void testAllMatchWithoutCombiner() {
+    SuperstepClasses classes =
+        new SuperstepClasses(IntNoOpComputation.class, null);
+    classes.verifyTypesMatch(createConfiguration(IntNoOpComputation.class));
+  }
+
+  @Test
+  public void testAllMatchWithCombiner() {
+    SuperstepClasses classes =
+        new SuperstepClasses(IntIntIntLongDoubleComputation.class,
+            IntDoubleCombiner.class);
+    classes.verifyTypesMatch(
+        createConfiguration(IntIntIntIntLongComputation.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testDifferentIdTypes() {
+    SuperstepClasses classes =
+        new SuperstepClasses(LongIntIntLongIntComputation.class, null);
+    classes.verifyTypesMatch(
+        createConfiguration(IntIntIntIntLongComputation.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testDifferentVertexValueTypes() {
+    SuperstepClasses classes =
+        new SuperstepClasses(IntLongIntLongIntComputation.class, null);
+    classes.verifyTypesMatch(
+        createConfiguration(IntIntIntIntLongComputation.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testDifferentEdgeDataTypes() {
+    SuperstepClasses classes =
+        new SuperstepClasses(IntIntLongLongIntComputation.class, null);
+    classes.verifyTypesMatch(
+        createConfiguration(IntIntIntIntLongComputation.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testDifferentMessageTypes() {
+    SuperstepClasses classes =
+        new SuperstepClasses(IntIntIntIntLongComputation.class, null);
+    classes.verifyTypesMatch(
+        createConfiguration(IntIntIntLongDoubleComputation.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testDifferentCombinerIdType() {
+    SuperstepClasses classes =
+        new SuperstepClasses(IntIntIntLongDoubleComputation.class,
+            DoubleDoubleCombiner.class);
+    classes.verifyTypesMatch(
+        createConfiguration(IntIntIntIntLongComputation.class));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testDifferentCombinerMessageType() {
+    SuperstepClasses classes =
+        new SuperstepClasses(IntIntIntLongDoubleComputation.class,
+            IntLongCombiner.class);
+    classes.verifyTypesMatch(
+        createConfiguration(IntIntIntIntLongComputation.class));
+  }
+
+  private static ImmutableClassesGiraphConfiguration createConfiguration(
+      Class<? extends Computation> computationClass) {
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setComputationClass(computationClass);
+    return new ImmutableClassesGiraphConfiguration(conf);
+  }
+
+  public static class NoOpComputation<I extends WritableComparable,
+      V extends Writable, E extends Writable, M1 extends Writable,
+      M2 extends Writable> extends Computation<I, V, E, M1, M2> {
+    @Override
+    public void compute(Vertex<I, V, E> vertex,
+        Iterable<M1> messages) throws IOException {
+    }
+  }
+
+  private static class IntIntIntIntLongComputation extends
+      NoOpComputation<IntWritable, IntWritable, IntWritable, IntWritable,
+          LongWritable> { }
+
+  private static class IntIntIntLongDoubleComputation extends
+      NoOpComputation<IntWritable, IntWritable, IntWritable, LongWritable,
+          DoubleWritable> { }
+
+  private static class LongIntIntLongIntComputation extends
+      NoOpComputation<LongWritable, IntWritable, IntWritable, LongWritable,
+          IntWritable> { }
+
+  private static class IntLongIntLongIntComputation extends
+      NoOpComputation<IntWritable, LongWritable, IntWritable, LongWritable,
+          IntWritable> { }
+
+  private static class IntIntLongLongIntComputation extends
+      NoOpComputation<IntWritable, IntWritable, LongWritable, LongWritable,
+          IntWritable> { }
+
+  private static class NoOpCombiner<I extends WritableComparable,
+      M extends Writable> extends Combiner<I, M> {
+    @Override
+    public void combine(I vertexIndex, M originalMessage, M messageToCombine) {
+    }
+
+    @Override
+    public M createInitialMessage() {
+      return null;
+    }
+  }
+
+  private static class IntDoubleCombiner extends NoOpCombiner<IntWritable,
+      DoubleWritable> { }
+
+  private static class DoubleDoubleCombiner extends NoOpCombiner<DoubleWritable,
+      DoubleWritable> { }
+
+  private static class IntLongCombiner extends NoOpCombiner<IntWritable,
+      LongWritable> { }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java b/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
index 9d6b215..0461624 100644
--- a/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
+++ b/giraph-core/src/test/java/org/apache/giraph/master/TestMasterObserver.java
@@ -18,14 +18,19 @@
 
 package org.apache.giraph.master;
 
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.edge.Edge;
 import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.formats.IntNullNullTextInputFormat;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -33,17 +38,45 @@ import java.io.IOException;
 import static org.apache.hadoop.util.StringUtils.arrayToString;
 import static org.junit.Assert.assertEquals;
 
-public class TestMasterObserver {
-  public static class NoOpVertex extends Vertex<IntWritable, NullWritable,
-      NullWritable, NullWritable> {
-    private int count = 0;
+import com.google.common.collect.ImmutableList;
 
+public class TestMasterObserver {
+  public static class SimpleComputation extends BasicComputation<IntWritable,
+      IntWritable, NullWritable, NullWritable> {
     @Override
-    public void compute(Iterable<NullWritable> messages) throws IOException {
-      if (count == 2) {
-        voteToHalt();
+    public void compute(
+        Vertex<IntWritable, IntWritable, NullWritable> vertex,
+        Iterable<NullWritable> messages) throws IOException {
+      int currentValue = vertex.getValue().get();
+      if (currentValue == 2) {
+        vertex.voteToHalt();
       }
-      ++count;
+      vertex.setValue(new IntWritable(currentValue + 1));
+    }
+  }
+
+  public static class InputFormat extends TextVertexInputFormat<
+      IntWritable, IntWritable, NullWritable> {
+    @Override
+    public TextVertexReader createVertexReader(
+        InputSplit split, TaskAttemptContext context) throws IOException {
+      return new TextVertexReaderFromEachLine() {
+        @Override
+        protected IntWritable getId(Text line) throws IOException {
+          return new IntWritable(Integer.parseInt(line.toString()));
+        }
+
+        @Override
+        protected IntWritable getValue(Text line) throws IOException {
+          return new IntWritable(0);
+        }
+
+        @Override
+        protected Iterable<Edge<IntWritable, NullWritable>> getEdges(
+            Text line) throws IOException {
+          return ImmutableList.of();
+        }
+      };
     }
   }
 
@@ -88,9 +121,9 @@ public class TestMasterObserver {
     GiraphConfiguration conf = new GiraphConfiguration();
     conf.set(GiraphConstants.MASTER_OBSERVER_CLASSES.getKey(),
         arrayToString(klasses));
-    conf.setVertexClass(NoOpVertex.class);
+    conf.setComputationClass(SimpleComputation.class);
     conf.setOutEdgesClass(ByteArrayEdges.class);
-    conf.setVertexInputFormatClass(IntNullNullTextInputFormat.class);
+    conf.setVertexInputFormatClass(InputFormat.class);
     InternalVertexRunner.run(conf, graph);
 
     assertEquals(2, Obs.preApp);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/master/TestSwitchClasses.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/master/TestSwitchClasses.java b/giraph-core/src/test/java/org/apache/giraph/master/TestSwitchClasses.java
new file mode 100644
index 0000000..4069972
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/master/TestSwitchClasses.java
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.master;
+
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.graph.Computation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.InternalVertexRunner;
+import org.apache.giraph.utils.TestGraph;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import junit.framework.Assert;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+
+/** Test switching Computation and Combiner class during application */
+public class TestSwitchClasses {
+  @Test
+  public void testSwitchingClasses() throws Exception {
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setComputationClass(Computation3.class);
+    conf.setMasterComputeClass(SwitchingClassesMasterCompute.class);
+
+    TestGraph<IntWritable, StatusValue, IntWritable> graph =
+        new TestGraph<IntWritable, StatusValue, IntWritable>(conf);
+    IntWritable id1 = new IntWritable(1);
+    graph.addVertex(id1, new StatusValue());
+    IntWritable id2 = new IntWritable(2);
+    graph.addVertex(id2, new StatusValue());
+    graph = InternalVertexRunner.run(conf, graph);
+
+    Assert.assertEquals(2, graph.getVertices().size());
+    StatusValue value1 = graph.getVertex(id1).getValue();
+    StatusValue value2 = graph.getVertex(id2).getValue();
+
+    // Check that computations were performed in expected order
+    ArrayList<Integer> expectedComputations = Lists.newArrayList(1, 1, 2, 3, 1);
+    checkComputations(expectedComputations, value1.computations);
+    checkComputations(expectedComputations, value2.computations);
+
+    // Check that messages were sent in the correct superstep,
+    // and combined when needed
+    ArrayList<HashSet<Double>> messages1 =
+        Lists.newArrayList(
+            Sets.<Double>newHashSet(),
+            Sets.<Double>newHashSet(11d),
+            Sets.<Double>newHashSet(11d),
+            Sets.<Double>newHashSet(101.5, 201.5),
+            Sets.<Double>newHashSet(3002d));
+    checkMessages(messages1, value1.messagesReceived);
+    ArrayList<HashSet<Double>> messages2 =
+        Lists.newArrayList(
+            Sets.<Double>newHashSet(),
+            Sets.<Double>newHashSet(12d),
+            Sets.<Double>newHashSet(12d),
+            Sets.<Double>newHashSet(102.5, 202.5),
+            Sets.<Double>newHashSet(3004d));
+    checkMessages(messages2, value2.messagesReceived);
+  }
+
+  private static void checkComputations(ArrayList<Integer> expected,
+      ArrayList<Integer> actual) {
+    Assert.assertEquals("Incorrect number of supersteps",
+        expected.size(), actual.size());
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals("Incorrect computation on superstep " + i,
+          (int) expected.get(i), (int) actual.get(i));
+    }
+  }
+
+  private static void checkMessages(ArrayList<HashSet<Double>> expected,
+      ArrayList<HashSet<Double>> actual) {
+    Assert.assertEquals(expected.size(), actual.size());
+    for (int i = 0; i < expected.size(); i++) {
+      Assert.assertEquals(expected.get(i).size(), actual.get(i).size());
+      for (Double value : expected.get(i)) {
+        Assert.assertTrue(actual.get(i).contains(value));
+      }
+    }
+  }
+
+  public static class SwitchingClassesMasterCompute
+      extends DefaultMasterCompute {
+    @Override
+    public void compute() {
+      switch ((int) getSuperstep()) {
+        case 0:
+          setComputation(Computation1.class);
+          setCombiner(MinimumCombiner.class);
+          break;
+        case 1:
+          // test classes don't change
+          break;
+        case 2:
+          setComputation(Computation2.class);
+          // test combiner removed
+          setCombiner(null);
+          break;
+        case 3:
+          setComputation(Computation3.class);
+          setCombiner(SumCombiner.class);
+          break;
+        case 4:
+          setComputation(Computation1.class);
+          break;
+        default:
+          haltComputation();
+      }
+    }
+  }
+
+  public static class Computation1 extends Computation<IntWritable,
+      StatusValue, IntWritable, IntWritable, IntWritable> {
+    @Override
+    public void compute(Vertex<IntWritable, StatusValue, IntWritable> vertex,
+        Iterable<IntWritable> messages) throws IOException {
+      vertex.getValue().computations.add(1);
+      vertex.getValue().addIntMessages(messages);
+
+      IntWritable otherId = new IntWritable(3 - vertex.getId().get());
+      sendMessage(otherId, new IntWritable(otherId.get() + 10));
+      sendMessage(otherId, new IntWritable(otherId.get() + 20));
+    }
+  }
+
+  public static class Computation2 extends Computation<IntWritable,
+      StatusValue, IntWritable, IntWritable, DoubleWritable> {
+    @Override
+    public void compute(Vertex<IntWritable, StatusValue, IntWritable> vertex,
+        Iterable<IntWritable> messages) throws IOException {
+      vertex.getValue().computations.add(2);
+      vertex.getValue().addIntMessages(messages);
+
+      IntWritable otherId = new IntWritable(3 - vertex.getId().get());
+      sendMessage(otherId, new DoubleWritable(otherId.get() + 100.5));
+      sendMessage(otherId, new DoubleWritable(otherId.get() + 200.5));
+    }
+  }
+
+  public static class Computation3 extends Computation<IntWritable,
+      StatusValue, IntWritable, DoubleWritable, IntWritable> {
+    @Override
+    public void compute(
+        Vertex<IntWritable, StatusValue, IntWritable> vertex,
+        Iterable<DoubleWritable> messages) throws IOException {
+      vertex.getValue().computations.add(3);
+      vertex.getValue().addDoubleMessages(messages);
+
+      IntWritable otherId = new IntWritable(3 - vertex.getId().get());
+      sendMessage(otherId, new IntWritable(otherId.get() + 1000));
+      sendMessage(otherId, new IntWritable(otherId.get() + 2000));
+    }
+  }
+
+  public static class MinimumCombiner extends Combiner<IntWritable,
+      IntWritable> {
+    @Override
+    public void combine(IntWritable vertexIndex, IntWritable originalMessage,
+        IntWritable messageToCombine) {
+      originalMessage.set(
+          Math.min(originalMessage.get(), messageToCombine.get()));
+    }
+
+    @Override
+    public IntWritable createInitialMessage() {
+      return new IntWritable(Integer.MAX_VALUE);
+    }
+  }
+
+  public static class SumCombiner extends Combiner<IntWritable, IntWritable> {
+    @Override
+    public void combine(IntWritable vertexIndex, IntWritable originalMessage,
+        IntWritable messageToCombine) {
+      originalMessage.set(originalMessage.get() + messageToCombine.get());
+    }
+
+    @Override
+    public IntWritable createInitialMessage() {
+      return new IntWritable(0);
+    }
+  }
+
+  public static class StatusValue implements Writable {
+    private ArrayList<Integer> computations = new ArrayList<Integer>();
+    private ArrayList<HashSet<Double>> messagesReceived =
+        new ArrayList<HashSet<Double>>();
+
+    public StatusValue() {
+    }
+
+    public void addIntMessages(Iterable<IntWritable> messages) {
+      HashSet<Double> messagesList = new HashSet<Double>();
+      for (IntWritable message : messages) {
+        messagesList.add((double) message.get());
+      }
+      messagesReceived.add(messagesList);
+    }
+
+    public void addDoubleMessages(Iterable<DoubleWritable> messages) {
+      HashSet<Double> messagesList = new HashSet<Double>();
+      for (DoubleWritable message : messages) {
+        messagesList.add(message.get());
+      }
+      messagesReceived.add(messagesList);
+    }
+
+    @Override
+    public void write(DataOutput dataOutput) throws IOException {
+      dataOutput.writeInt(computations.size());
+      for (Integer computation : computations) {
+        dataOutput.writeInt(computation);
+      }
+      dataOutput.writeInt(messagesReceived.size());
+      for (HashSet<Double> messages : messagesReceived) {
+        dataOutput.writeInt(messages.size());
+        for (Double message : messages) {
+          dataOutput.writeDouble(message);
+        }
+      }
+    }
+
+    @Override
+    public void readFields(DataInput dataInput) throws IOException {
+      int size = dataInput.readInt();
+      computations = new ArrayList<Integer>(size);
+      for (int i = 0; i < size; i++) {
+        computations.add(dataInput.readInt());
+      }
+      size = dataInput.readInt();
+      messagesReceived = new ArrayList<HashSet<Double>>(size);
+      for (int i = 0; i < size; i++) {
+        int size2 = dataInput.readInt();
+        HashSet<Double> messages = new HashSet<Double>(size2);
+        for (int j = 0; j < size2; j++) {
+          messages.add(dataInput.readDouble());
+        }
+        messagesReceived.add(messages);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java b/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
index 9b655af..8a1e9ed 100644
--- a/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
+++ b/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
@@ -23,6 +23,7 @@ import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.GiraphTransferRegulator;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
@@ -44,16 +45,14 @@ public class TestGiraphTransferRegulator {
   /** Job filled in by setup() */
   private GiraphJob job;
   /** Instantiated vertex filled in from setup() */
-  private TestVertex vertex = new TestVertex();
+  private Vertex<IntWritable, FloatWritable, DoubleWritable>
+      vertex = new Vertex<IntWritable, FloatWritable, DoubleWritable>();
 
   /**
    * Dummy vertex.
    */
-  public static class TestVertex extends
-      Vertex<IntWritable, FloatWritable, DoubleWritable, LongWritable> {
-    @Override
-    public void compute(Iterable<LongWritable> messages) throws IOException { }
-  }
+  public static class TestComputation extends NoOpComputation<IntWritable,
+      FloatWritable, DoubleWritable, LongWritable> { }
 
   @Before
   public void setUp() {
@@ -62,7 +61,7 @@ public class TestGiraphTransferRegulator {
     } catch (IOException e) {
       throw new RuntimeException("setUp: Failed", e);
     }
-    job.getConfiguration().setVertexClass(TestVertex.class);
+    job.getConfiguration().setComputationClass(TestComputation.class);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java b/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
index 5a93d41..45542b9 100644
--- a/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
+++ b/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
@@ -24,6 +24,7 @@ import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.NoOpComputation;
 import org.apache.giraph.utils.UnsafeByteArrayInputStream;
 import org.apache.giraph.utils.UnsafeByteArrayOutputStream;
 import org.apache.hadoop.io.IntWritable;
@@ -48,25 +49,20 @@ import static org.mockito.Mockito.mock;
  */
 public class TestPartitionStores {
   private ImmutableClassesGiraphConfiguration<IntWritable, IntWritable,
-      NullWritable, IntWritable> conf;
+      NullWritable> conf;
   private Mapper<?, ?, ?, ?>.Context context;
 
-  public static class MyVertex extends Vertex<IntWritable, IntWritable,
-      NullWritable, IntWritable> {
-    @Override
-    public void compute(Iterable<IntWritable> messages) throws IOException {}
-  }
+  public static class MyComputation extends NoOpComputation<IntWritable,
+      IntWritable, NullWritable, IntWritable> { }
 
-  private Partition<IntWritable, IntWritable, NullWritable,
-        IntWritable> createPartition(
+  private Partition<IntWritable, IntWritable, NullWritable> createPartition(
       ImmutableClassesGiraphConfiguration<IntWritable, IntWritable,
-          NullWritable, IntWritable> conf,
+          NullWritable> conf,
       Integer id,
-      Vertex<IntWritable, IntWritable, NullWritable,
-          IntWritable>... vertices) {
-    Partition<IntWritable, IntWritable, NullWritable, IntWritable> partition =
+      Vertex<IntWritable, IntWritable, NullWritable>... vertices) {
+    Partition<IntWritable, IntWritable, NullWritable> partition =
         conf.createPartition(id, context);
-    for (Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v :
+    for (Vertex<IntWritable, IntWritable, NullWritable> v :
         vertices) {
       partition.putVertex(v);
     }
@@ -76,17 +72,17 @@ public class TestPartitionStores {
   @Before
   public void setUp() {
     GiraphConfiguration configuration = new GiraphConfiguration();
-    configuration.setVertexClass(MyVertex.class);
+    configuration.setComputationClass(MyComputation.class);
     conf = new ImmutableClassesGiraphConfiguration<IntWritable, IntWritable,
-        NullWritable, IntWritable>(configuration);
+        NullWritable>(configuration);
     context = mock(Mapper.Context.class);
   }
 
   @Test
   public void testSimplePartitionStore() {
-    PartitionStore<IntWritable, IntWritable, NullWritable, IntWritable>
+    PartitionStore<IntWritable, IntWritable, NullWritable>
         partitionStore = new SimplePartitionStore<IntWritable, IntWritable,
-                NullWritable, IntWritable>(conf, context);
+                NullWritable>(conf, context);
     testReadWrite(partitionStore, conf);
     partitionStore.shutdown();
   }
@@ -94,30 +90,23 @@ public class TestPartitionStores {
   @Test
   public void testUnsafePartitionSerializationClass() throws IOException {
     conf.setPartitionClass(ByteArrayPartition.class);
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v1 =
+    Vertex<IntWritable, IntWritable, NullWritable> v1 =
         conf.createVertex();
     v1.initialize(new IntWritable(1), new IntWritable(1));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v2 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v2 = conf.createVertex();
     v2.initialize(new IntWritable(2), new IntWritable(2));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v3 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v3 = conf.createVertex();
     v3.initialize(new IntWritable(3), new IntWritable(3));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v4 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v4 = conf.createVertex();
     v4.initialize(new IntWritable(4), new IntWritable(4));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v5 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v5 = conf.createVertex();
     v5.initialize(new IntWritable(5), new IntWritable(5));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v6 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v6 = conf.createVertex();
     v6.initialize(new IntWritable(6), new IntWritable(6));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v7 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v7 = conf.createVertex();
     v7.initialize(new IntWritable(7), new IntWritable(7));
 
-    Partition<IntWritable, IntWritable, NullWritable,
-        IntWritable> partition =
+    Partition<IntWritable, IntWritable, NullWritable> partition =
         createPartition(conf, 3, v1, v2, v3, v4, v5, v6, v7);
     assertEquals(3, partition.getId());
     assertEquals(0, partition.getEdgeCount());
@@ -127,9 +116,8 @@ public class TestPartitionStores {
     partition.write(outputStream);
     UnsafeByteArrayInputStream inputStream = new UnsafeByteArrayInputStream(
         outputStream.getByteArray(), 0, outputStream.getPos());
-    Partition<IntWritable, IntWritable, NullWritable,
-        IntWritable> deserializatedPartition = conf.createPartition(-1,
-        context);
+    Partition<IntWritable, IntWritable, NullWritable> deserializatedPartition =
+        conf.createPartition(-1, context);
     deserializatedPartition.readFields(inputStream);
 
     assertEquals(3, deserializatedPartition.getId());
@@ -146,9 +134,9 @@ public class TestPartitionStores {
     GiraphConstants.MAX_PARTITIONS_IN_MEMORY.set(conf, 1);
     conf.setPartitionClass(ByteArrayPartition.class);
     
-    PartitionStore<IntWritable, IntWritable, NullWritable, IntWritable>
-        partitionStore = new DiskBackedPartitionStore<IntWritable,
-            IntWritable, NullWritable, IntWritable>(conf, context);
+    PartitionStore<IntWritable, IntWritable, NullWritable> partitionStore =
+        new DiskBackedPartitionStore<IntWritable, IntWritable, NullWritable>(
+            conf, context);
     testReadWrite(partitionStore, conf);
     partitionStore.shutdown();
     FileUtils.deleteDirectory(directory);
@@ -162,15 +150,15 @@ public class TestPartitionStores {
     GiraphConstants.USE_OUT_OF_CORE_GRAPH.set(conf, true);
     GiraphConstants.MAX_PARTITIONS_IN_MEMORY.set(conf, 1);
 
-    PartitionStore<IntWritable, IntWritable, NullWritable, IntWritable>
-        partitionStore = new DiskBackedPartitionStore<IntWritable,
-                        IntWritable, NullWritable, IntWritable>(conf, context);
+    PartitionStore<IntWritable, IntWritable, NullWritable> partitionStore =
+        new DiskBackedPartitionStore<IntWritable, IntWritable, NullWritable>(
+            conf, context);
     testReadWrite(partitionStore, conf);
     partitionStore.shutdown();
 
     GiraphConstants.MAX_PARTITIONS_IN_MEMORY.set(conf, 2);
     partitionStore = new DiskBackedPartitionStore<IntWritable,
-            IntWritable, NullWritable, IntWritable>(conf, context);
+            IntWritable, NullWritable>(conf, context);
     testReadWrite(partitionStore, conf);
     partitionStore.shutdown();
     FileUtils.deleteDirectory(directory);
@@ -184,29 +172,22 @@ public class TestPartitionStores {
    */
   public void testReadWrite(
       PartitionStore<IntWritable, IntWritable,
-          NullWritable, IntWritable> partitionStore,
+          NullWritable> partitionStore,
       ImmutableClassesGiraphConfiguration<IntWritable, IntWritable,
-          NullWritable, IntWritable> conf) {
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v1 =
-        conf.createVertex();
+          NullWritable> conf) {
+    Vertex<IntWritable, IntWritable, NullWritable> v1 = conf.createVertex();
     v1.initialize(new IntWritable(1), new IntWritable(1));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v2 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v2 = conf.createVertex();
     v2.initialize(new IntWritable(2), new IntWritable(2));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v3 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v3 = conf.createVertex();
     v3.initialize(new IntWritable(3), new IntWritable(3));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v4 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v4 = conf.createVertex();
     v4.initialize(new IntWritable(4), new IntWritable(4));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v5 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v5 = conf.createVertex();
     v5.initialize(new IntWritable(5), new IntWritable(5));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v6 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v6 = conf.createVertex();
     v6.initialize(new IntWritable(7), new IntWritable(7));
-    Vertex<IntWritable, IntWritable, NullWritable, IntWritable> v7 =
-        conf.createVertex();
+    Vertex<IntWritable, IntWritable, NullWritable> v7 = conf.createVertex();
     v7.initialize(new IntWritable(7), new IntWritable(7));
     v7.addEdge(EdgeFactory.create(new IntWritable(1)));
     v7.addEdge(EdgeFactory.create(new IntWritable(2)));
@@ -218,15 +199,15 @@ public class TestPartitionStores {
     partitionStore.addPartition(createPartition(conf, 1, v6));
     partitionStore.addPartition(createPartition(conf, 4, v7));
 
-    Partition<IntWritable, IntWritable, NullWritable, IntWritable> partition1 =
+    Partition<IntWritable, IntWritable, NullWritable> partition1 =
         partitionStore.getPartition(1);
     partitionStore.putPartition(partition1);
-    Partition<IntWritable, IntWritable, NullWritable, IntWritable> partition2 =
+    Partition<IntWritable, IntWritable, NullWritable> partition2 =
         partitionStore.getPartition(2);
     partitionStore.putPartition(partition2);
-    Partition<IntWritable, IntWritable, NullWritable,
-        IntWritable> partition3 = partitionStore.removePartition(3);
-    Partition<IntWritable, IntWritable, NullWritable, IntWritable> partition4 =
+    Partition<IntWritable, IntWritable, NullWritable> partition3 =
+        partitionStore.removePartition(3);
+    Partition<IntWritable, IntWritable, NullWritable> partition4 =
         partitionStore.getPartition(4);
     partitionStore.putPartition(partition4);
 
@@ -234,12 +215,12 @@ public class TestPartitionStores {
     assertEquals(3, Iterables.size(partitionStore.getPartitionIds()));
     int partitionsNumber = 0;
     for (Integer partitionId : partitionStore.getPartitionIds()) {
-      Partition<IntWritable, IntWritable, NullWritable, IntWritable> p = 
+      Partition<IntWritable, IntWritable, NullWritable> p =
           partitionStore.getPartition(partitionId);
       partitionStore.putPartition(p);
       partitionsNumber++;
     }
-    Partition<IntWritable, IntWritable, NullWritable, IntWritable> partition;
+    Partition<IntWritable, IntWritable, NullWritable> partition;
     assertEquals(3, partitionsNumber);
     assertTrue(partitionStore.hasPartition(1));
     assertTrue(partitionStore.hasPartition(2));

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/utils/ComputationCountEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/ComputationCountEdges.java b/giraph-core/src/test/java/org/apache/giraph/utils/ComputationCountEdges.java
new file mode 100644
index 0000000..ba05b66
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/ComputationCountEdges.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.utils;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import java.io.IOException;
+
+/**
+ * Computation which sets the value of vertex to its number of edges,
+ * used for testing. Vertex ids, values, edge values and messages are all
+ * integers.
+ */
+public class ComputationCountEdges extends BasicComputation<IntWritable,
+    IntWritable, NullWritable, NullWritable> {
+  @Override
+  public void compute(Vertex<IntWritable, IntWritable, NullWritable> vertex,
+      Iterable<NullWritable> messages) throws IOException {
+    vertex.setValue(new IntWritable(vertex.getNumEdges()));
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/utils/IntIntNullNoOpComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/IntIntNullNoOpComputation.java b/giraph-core/src/test/java/org/apache/giraph/utils/IntIntNullNoOpComputation.java
new file mode 100644
index 0000000..c628b0b
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/IntIntNullNoOpComputation.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.utils;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+
+/**
+ * Computation which does nothing, used for testing. Vertex ids and values
+ * are integers, edge values and messages are nulls.
+ */
+public class IntIntNullNoOpComputation extends
+    NoOpComputation<IntWritable, IntWritable, NullWritable, NullWritable> {
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/utils/IntNoOpComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/IntNoOpComputation.java b/giraph-core/src/test/java/org/apache/giraph/utils/IntNoOpComputation.java
new file mode 100644
index 0000000..e2fbb39
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/IntNoOpComputation.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.utils;
+
+import org.apache.hadoop.io.IntWritable;
+
+/**
+ * Computation which does nothing, used for testing. Vertex ids, values,
+ * edge values and messages are all integers.
+ */
+public class IntNoOpComputation extends NoOpComputation<IntWritable,
+    IntWritable, IntWritable, IntWritable> {
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/utils/LongNoOpComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/LongNoOpComputation.java b/giraph-core/src/test/java/org/apache/giraph/utils/LongNoOpComputation.java
new file mode 100644
index 0000000..c83348f
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/LongNoOpComputation.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.utils;
+
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Computation which does nothing, used for testing. Vertex ids, values,
+ * edge values and messages are all longs.
+ */
+public class LongNoOpComputation extends NoOpComputation<LongWritable,
+    LongWritable, LongWritable, LongWritable> {
+}


[08/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
index f00116a..bd48116 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java
@@ -30,13 +30,13 @@ import org.apache.giraph.comm.netty.NettyMasterServer;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.counters.GiraphStats;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.InputSplitPaths;
 import org.apache.giraph.graph.GlobalStats;
 import org.apache.giraph.graph.AddressesAndPartitionsWritable;
 import org.apache.giraph.graph.GraphFunctions;
 import org.apache.giraph.graph.InputSplitEvents;
 import org.apache.giraph.bsp.BspService;
+import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.GiraphInputFormat;
 import org.apache.giraph.graph.GraphTaskManager;
@@ -116,13 +116,12 @@ import static org.apache.giraph.conf.GiraphConstants.USE_INPUT_SPLIT_LOCALITY;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class BspServiceMaster<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends BspService<I, V, E, M>
-    implements CentralizedServiceMaster<I, V, E, M>,
+    V extends Writable, E extends Writable>
+    extends BspService<I, V, E>
+    implements CentralizedServiceMaster<I, V, E>,
     ResetSuperstepMetricsObserver {
   /** Print worker names only if there are 10 workers left */
   public static final int MAX_PRINTABLE_REMAINING_WORKERS = 10;
@@ -158,7 +157,7 @@ public class BspServiceMaster<I extends WritableComparable,
   /** State of the superstep changed */
   private final BspEvent superstepStateChanged;
   /** Master graph partitioner */
-  private final MasterGraphPartitioner<I, V, E, M> masterGraphPartitioner;
+  private final MasterGraphPartitioner<I, V, E> masterGraphPartitioner;
   /** All the partition stats from the last superstep */
   private final List<PartitionStats> allPartitionStatsList =
       new ArrayList<PartitionStats>();
@@ -195,14 +194,15 @@ public class BspServiceMaster<I extends WritableComparable,
       String serverPortList,
       int sessionMsecTimeout,
       Mapper<?, ?, ?, ?>.Context context,
-      GraphTaskManager<I, V, E, M> graphTaskManager) {
+      GraphTaskManager<I, V, E> graphTaskManager) {
     super(serverPortList, sessionMsecTimeout, context, graphTaskManager);
     workerWroteCheckpoint = new PredicateLock(context);
     registerBspEvent(workerWroteCheckpoint);
     superstepStateChanged = new PredicateLock(context);
     registerBspEvent(superstepStateChanged);
 
-    ImmutableClassesGiraphConfiguration<I, V, E, M> conf = getConfiguration();
+    ImmutableClassesGiraphConfiguration<I, V, E> conf =
+        getConfiguration();
 
     maxWorkers = conf.getMaxWorkers();
     minWorkers = conf.getMinWorkers();
@@ -743,6 +743,9 @@ public class BspServiceMaster<I extends WritableComparable,
     GlobalStats globalStats = new GlobalStats();
     globalStats.readFields(finalizedStream);
     updateCounters(globalStats);
+    SuperstepClasses superstepClasses = new SuperstepClasses();
+    superstepClasses.readFields(finalizedStream);
+    getConfiguration().updateSuperstepClasses(superstepClasses);
     int prefixFileCount = finalizedStream.readInt();
     for (int i = 0; i < prefixFileCount; ++i) {
       String metadataFilePath =
@@ -856,10 +859,11 @@ public class BspServiceMaster<I extends WritableComparable,
         if (masterChildArr.get(0).equals(myBid)) {
           GiraphStats.getInstance().getCurrentMasterTaskPartition().
               setValue(getTaskPartition());
-          masterCompute = getConfiguration().createMasterCompute();
           aggregatorHandler = new MasterAggregatorHandler(getConfiguration(),
               getContext());
           aggregatorHandler.initialize(this);
+          masterCompute = getConfiguration().createMasterCompute();
+          masterCompute.setMasterAggregatorUsage(aggregatorHandler);
 
           masterInfo = new MasterInfo();
           masterServer =
@@ -1006,6 +1010,7 @@ public class BspServiceMaster<I extends WritableComparable,
 
     // Format:
     // <global statistics>
+    // <superstep classes>
     // <number of files>
     // <used file prefix 0><used file prefix 1>...
     // <aggregator data>
@@ -1046,7 +1051,7 @@ public class BspServiceMaster<I extends WritableComparable,
   private void assignPartitionOwners(
       List<PartitionStats> allPartitionStatsList,
       List<WorkerInfo> chosenWorkerInfoList,
-      MasterGraphPartitioner<I, V, E, M> masterGraphPartitioner) {
+      MasterGraphPartitioner<I, V, E> masterGraphPartitioner) {
     Collection<PartitionOwner> partitionOwners;
     if (getSuperstep() == INPUT_SUPERSTEP ||
         getSuperstep() == getRestartedSuperstep()) {
@@ -1511,7 +1516,7 @@ public class BspServiceMaster<I extends WritableComparable,
     // Collect aggregator values, then run the master.compute() and
     // finally save the aggregator values
     aggregatorHandler.prepareSuperstep(masterClient);
-    runMasterCompute(getSuperstep());
+    SuperstepClasses superstepClasses = runMasterCompute(getSuperstep());
 
     // If the master is halted or all the vertices voted to halt and there
     // are no more messages in the system, stop the computation
@@ -1535,12 +1540,15 @@ public class BspServiceMaster<I extends WritableComparable,
       globalStats.setHaltComputation(true);
     }
 
+    superstepClasses.verifyTypesMatch(getConfiguration());
+    getConfiguration().updateSuperstepClasses(superstepClasses);
+
     // Let everyone know the aggregated application state through the
     // superstep finishing znode.
     String superstepFinishedNode =
         getSuperstepFinishedPath(getApplicationAttempt(), getSuperstep());
     WritableUtils.writeToZnode(
-        getZkExt(), superstepFinishedNode, -1, globalStats);
+        getZkExt(), superstepFinishedNode, -1, globalStats, superstepClasses);
     updateCounters(globalStats);
 
     cleanUpOldSuperstep(getSuperstep() - 1);
@@ -1564,16 +1572,19 @@ public class BspServiceMaster<I extends WritableComparable,
    * Run the master.compute() class
    *
    * @param superstep superstep for which to run the master.compute()
+   * @return Superstep classes set by Master compute
    */
-  private void runMasterCompute(long superstep) {
+  private SuperstepClasses runMasterCompute(long superstep) {
     // The master.compute() should run logically before the workers, so
     // increase the superstep counter it uses by one
-    GraphState<I, V, E, M> graphState =
-        new GraphState<I, V, E, M>(superstep + 1,
-            GiraphStats.getInstance().getVertices().getValue(),
-            GiraphStats.getInstance().getEdges().getValue(),
-            getContext(), getGraphTaskManager(), null, null);
+    GraphState graphState = new GraphState(superstep + 1,
+        GiraphStats.getInstance().getVertices().getValue(),
+        GiraphStats.getInstance().getEdges().getValue(),
+        getContext());
+    SuperstepClasses superstepClasses =
+        new SuperstepClasses(getConfiguration());
     masterCompute.setGraphState(graphState);
+    masterCompute.setSuperstepClasses(superstepClasses);
     if (superstep == INPUT_SUPERSTEP) {
       try {
         masterCompute.initialize();
@@ -1590,6 +1601,7 @@ public class BspServiceMaster<I extends WritableComparable,
     GiraphTimerContext timerContext = masterComputeTimer.time();
     masterCompute.compute();
     timerContext.stop();
+    return superstepClasses;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
index f769c3a..0537915 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterAggregatorHandler.java
@@ -59,7 +59,7 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
   /** Progressable used to report progress */
   private final Progressable progressable;
   /** Giraph configuration */
-  private final ImmutableClassesGiraphConfiguration<?, ?, ?, ?> conf;
+  private final ImmutableClassesGiraphConfiguration<?, ?, ?> conf;
 
   /**
    * Constructor
@@ -68,7 +68,7 @@ public class MasterAggregatorHandler implements MasterAggregatorUsage,
    * @param progressable Progressable used for reporting progress
    */
   public MasterAggregatorHandler(
-      ImmutableClassesGiraphConfiguration<?, ?, ?, ?> conf,
+      ImmutableClassesGiraphConfiguration<?, ?, ?> conf,
       Progressable progressable) {
     this.conf = conf;
     this.progressable = progressable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
index 1c36e03..1f244bb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterCompute.java
@@ -18,9 +18,10 @@
 
 package org.apache.giraph.master;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.graph.GraphState;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -36,20 +37,19 @@ import org.apache.hadoop.mapreduce.Mapper;
  * master.compute() is called. This means aggregator values used by the workers
  * are consistent with aggregator values from the master from the same
  * superstep and aggregator used by the master are consistent with aggregator
- * values from the workers from the previous superstep. Note that the master
- * has to register its own aggregators (it does not call {@link WorkerContext}
- * functions), but it uses all aggregators by default, so useAggregator does
- * not have to be called.
+ * values from the workers from the previous superstep.
  */
-@SuppressWarnings("rawtypes")
-public abstract class MasterCompute implements MasterAggregatorUsage, Writable,
-    ImmutableClassesGiraphConfigurable {
+public abstract class MasterCompute
+    extends DefaultImmutableClassesGiraphConfigurable
+    implements MasterAggregatorUsage, Writable {
   /** If true, do not do anymore computation on this vertex. */
   private boolean halt = false;
-  /** Global graph state **/
+  /** Master aggregator usage */
+  private MasterAggregatorUsage masterAggregatorUsage;
+  /** Graph state */
   private GraphState graphState;
-  /** Configuration */
-  private ImmutableClassesGiraphConfiguration conf;
+  /** Computation and Combiner class used, which can be switched by master */
+  private SuperstepClasses superstepClasses;
 
   /**
    * Must be defined by user to specify what the master has to do.
@@ -68,8 +68,8 @@ public abstract class MasterCompute implements MasterAggregatorUsage, Writable,
    *
    * @return Current superstep
    */
-  public long getSuperstep() {
-    return getGraphState().getSuperstep();
+  public final long getSuperstep() {
+    return graphState.getSuperstep();
   }
 
   /**
@@ -78,8 +78,8 @@ public abstract class MasterCompute implements MasterAggregatorUsage, Writable,
    *
    * @return Total number of vertices (-1 if first superstep)
    */
-  public long getTotalNumVertices() {
-    return getGraphState().getTotalNumVertices();
+  public final long getTotalNumVertices() {
+    return graphState.getTotalNumVertices();
   }
 
   /**
@@ -88,15 +88,15 @@ public abstract class MasterCompute implements MasterAggregatorUsage, Writable,
    *
    * @return Total number of edges (-1 if first superstep)
    */
-  public long getTotalNumEdges() {
-    return getGraphState().getTotalNumEdges();
+  public final long getTotalNumEdges() {
+    return graphState.getTotalNumEdges();
   }
 
   /**
    * After this is called, the computation will stop, even if there are
    * still messages in the system or vertices that have not voted to halt.
    */
-  public void haltComputation() {
+  public final void haltComputation() {
     halt = true;
   }
 
@@ -105,43 +105,43 @@ public abstract class MasterCompute implements MasterAggregatorUsage, Writable,
    *
    * @return True if halted, false otherwise.
    */
-  public boolean isHalted() {
+  public final boolean isHalted() {
     return halt;
   }
 
   /**
-   * Get the graph state for all workers.
+   * Get the mapper context
    *
-   * @return Graph state for all workers
+   * @return Mapper context
    */
-  GraphState getGraphState() {
-    return graphState;
+  public final Mapper.Context getContext() {
+    return graphState.getContext();
   }
 
   /**
-   * Set the graph state for all workers
+   * Set Computation class to be used
    *
-   * @param graphState Graph state for all workers
+   * @param computationClass Computation class
    */
-  void setGraphState(GraphState graphState) {
-    this.graphState = graphState;
+  public final void setComputation(
+      Class<? extends Computation> computationClass) {
+    superstepClasses.setComputationClass(computationClass);
   }
 
   /**
-   * Get the mapper context
+   * Set Combiner class to be used
    *
-   * @return Mapper context
+   * @param combinerClass Combiner class
    */
-  public Mapper.Context getContext() {
-    return getGraphState().getContext();
+  public final void setCombiner(Class<? extends Combiner> combinerClass) {
+    superstepClasses.setCombinerClass(combinerClass);
   }
 
   @Override
   public final <A extends Writable> boolean registerAggregator(
     String name, Class<? extends Aggregator<A>> aggregatorClass)
     throws InstantiationException, IllegalAccessException {
-    return getGraphState().getGraphTaskManager().getMasterAggregatorUsage().
-        registerAggregator(name, aggregatorClass);
+    return masterAggregatorUsage.registerAggregator(name, aggregatorClass);
   }
 
   @Override
@@ -149,29 +149,31 @@ public abstract class MasterCompute implements MasterAggregatorUsage, Writable,
       String name,
       Class<? extends Aggregator<A>> aggregatorClass) throws
       InstantiationException, IllegalAccessException {
-    return getGraphState().getGraphTaskManager().getMasterAggregatorUsage().
-        registerPersistentAggregator(name, aggregatorClass);
+    return masterAggregatorUsage.registerPersistentAggregator(
+        name, aggregatorClass);
   }
 
   @Override
-  public <A extends Writable> A getAggregatedValue(String name) {
-    return getGraphState().getGraphTaskManager().getMasterAggregatorUsage().
-        <A>getAggregatedValue(name);
+  public final <A extends Writable> A getAggregatedValue(String name) {
+    return masterAggregatorUsage.<A>getAggregatedValue(name);
   }
 
   @Override
-  public <A extends Writable> void setAggregatedValue(String name, A value) {
-    getGraphState().getGraphTaskManager().getMasterAggregatorUsage().
-        setAggregatedValue(name, value);
+  public final <A extends Writable> void setAggregatedValue(
+      String name, A value) {
+    masterAggregatorUsage.setAggregatedValue(name, value);
   }
 
-  @Override
-  public ImmutableClassesGiraphConfiguration getConf() {
-    return conf;
+  final void setGraphState(GraphState graphState) {
+    this.graphState = graphState;
   }
 
-  @Override
-  public void setConf(ImmutableClassesGiraphConfiguration conf) {
-    this.conf = conf;
+  final void setMasterAggregatorUsage(MasterAggregatorUsage
+      masterAggregatorUsage) {
+    this.masterAggregatorUsage = masterAggregatorUsage;
+  }
+
+  final void setSuperstepClasses(SuperstepClasses superstepClasses) {
+    this.superstepClasses = superstepClasses;
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java b/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
index ba2f8eb..e8eeeed 100644
--- a/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
+++ b/giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java
@@ -43,17 +43,16 @@ import static org.apache.giraph.conf.GiraphConstants.USE_SUPERSTEP_COUNTERS;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class MasterThread<I extends WritableComparable, V extends Writable,
-    E extends Writable, M extends Writable> extends Thread {
+    E extends Writable> extends Thread {
   /** Counter group name for the Giraph timers */
   public static final String GIRAPH_TIMERS_COUNTER_GROUP_NAME = "Giraph Timers";
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(MasterThread.class);
   /** Reference to shared BspService */
-  private CentralizedServiceMaster<I, V, E, M> bspServiceMaster = null;
+  private CentralizedServiceMaster<I, V, E> bspServiceMaster = null;
   /** Context (for counters) */
   private final Context context;
   /** Use superstep counters? */
@@ -71,7 +70,7 @@ public class MasterThread<I extends WritableComparable, V extends Writable,
    *        been called.
    * @param context Context from the Mapper.
    */
-  public MasterThread(CentralizedServiceMaster<I, V, E, M> bspServiceMaster,
+  public MasterThread(CentralizedServiceMaster<I, V, E> bspServiceMaster,
       Context context) {
     super(MasterThread.class.getName());
     this.bspServiceMaster = bspServiceMaster;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/master/SuperstepClasses.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/master/SuperstepClasses.java b/giraph-core/src/main/java/org/apache/giraph/master/SuperstepClasses.java
new file mode 100644
index 0000000..a12ef58
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/master/SuperstepClasses.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.master;
+
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.graph.Computation;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.hadoop.io.Writable;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Modifier;
+import java.util.List;
+
+/**
+ * Holds Computation and Combiner class.
+ */
+public class SuperstepClasses implements Writable {
+  /** Computation class to be used in the following superstep */
+  private Class<? extends Computation> computationClass;
+  /** Combiner class to be used in the following superstep */
+  private Class<? extends Combiner> combinerClass;
+
+  /**
+   * Default constructor
+   */
+  public SuperstepClasses() {
+  }
+
+  /**
+   * Constructor
+   *
+   * @param conf Configuration
+   */
+  @SuppressWarnings("unchecked")
+  public SuperstepClasses(ImmutableClassesGiraphConfiguration conf) {
+    this(conf.getComputationClass(), conf.getCombinerClass());
+  }
+
+  /**
+   * Constructor
+   *
+   * @param computationClass Computation class
+   * @param combinerClass Combiner class
+   */
+  public SuperstepClasses(Class<? extends Computation> computationClass,
+      Class<? extends Combiner> combinerClass) {
+    this.computationClass = computationClass;
+    this.combinerClass = combinerClass;
+  }
+
+  public Class<? extends Computation> getComputationClass() {
+    return computationClass;
+  }
+
+  public Class<? extends Combiner> getCombinerClass() {
+    return combinerClass;
+  }
+
+  public void setComputationClass(
+      Class<? extends Computation> computationClass) {
+    this.computationClass = computationClass;
+  }
+
+  public void setCombinerClass(Class<? extends Combiner> combinerClass) {
+    this.combinerClass = combinerClass;
+  }
+
+  /**
+   * Verify that types of current Computation and Combiner are valid. If types
+   * don't match an {@link IllegalStateException} will be thrown.
+   *
+   * @param conf Configuration to verify this with
+   */
+  public void verifyTypesMatch(ImmutableClassesGiraphConfiguration conf) {
+    List<Class<?>> computationTypes = ReflectionUtils.getTypeArguments(
+        Computation.class, computationClass);
+    verifyTypes(conf.getVertexIdClass(), computationTypes.get(0),
+        "Vertex id", computationClass);
+    verifyTypes(conf.getVertexValueClass(), computationTypes.get(1),
+        "Vertex value", computationClass);
+    verifyTypes(conf.getEdgeValueClass(), computationTypes.get(2),
+        "Edge value", computationClass);
+    verifyTypes(conf.getOutgoingMessageValueClass(), computationTypes.get(3),
+        "Previous outgoing and new incoming message", computationClass);
+    Class<?> outgoingMessageType = computationTypes.get(4);
+    if (outgoingMessageType.isInterface()) {
+      throw new IllegalStateException("verifyTypesMatch: " +
+          "Message type must be concrete class " + outgoingMessageType);
+    }
+    if (Modifier.isAbstract(outgoingMessageType.getModifiers())) {
+      throw new IllegalStateException("verifyTypesMatch: " +
+          "Message type can't be abstract class" + outgoingMessageType);
+    }
+    if (combinerClass != null) {
+      List<Class<?>> combinerTypes = ReflectionUtils.getTypeArguments(
+          Combiner.class, combinerClass);
+      verifyTypes(conf.getVertexIdClass(), combinerTypes.get(0),
+          "Vertex id", combinerClass);
+      verifyTypes(outgoingMessageType, combinerTypes.get(1),
+          "Outgoing message", combinerClass);
+    }
+  }
+
+  /**
+   * Verify that found type matches the expected type. If types don't match an
+   * {@link IllegalStateException} will be thrown.
+   *
+   * @param expected Expected type
+   * @param actual Actual type
+   * @param typeDesc String description of the type (for exception description)
+   * @param mainClass Class in which the actual type was found (for exception
+   *                  description)
+   */
+  private void verifyTypes(Class<?> expected, Class<?> actual,
+      String typeDesc, Class<?> mainClass) {
+    if (!expected.equals(actual)) {
+      throw new IllegalStateException("verifyTypes: " + typeDesc + " types " +
+          "don't match, in " + mainClass.getName() + " " + expected +
+          " expected, but " + actual + " found");
+    }
+  }
+
+  @Override
+  public void write(DataOutput output) throws IOException {
+    WritableUtils.writeClass(computationClass, output);
+    WritableUtils.writeClass(combinerClass, output);
+  }
+
+  @Override
+  public void readFields(DataInput input) throws IOException {
+    computationClass = WritableUtils.readClass(input);
+    combinerClass = WritableUtils.readClass(input);
+  }
+
+  @Override
+  public String toString() {
+    return "(computation=" + computationClass.getName() + ",combiner=" +
+        ((combinerClass == null) ? "null" : combinerClass.getName()) + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartition.java b/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartition.java
index dc9192e..f2b8552 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/BasicPartition.java
@@ -34,35 +34,31 @@ import java.io.IOException;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public abstract class BasicPartition<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements Partition<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    implements Partition<I, V, E> {
   /** Configuration from the worker */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Partition id */
   private int id;
   /** Context used to report progress */
   private Progressable progressable;
-  /** Partition context */
-  private PartitionContext partitionContext;
 
   @Override
   public void initialize(int partitionId, Progressable progressable) {
     setId(partitionId);
     setProgressable(progressable);
-    partitionContext = conf.createPartitionContext();
   }
 
   @Override
   public void setConf(
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration) {
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration) {
     conf = configuration;
   }
 
   @Override
-  public ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
+  public ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
     return conf;
   }
 
@@ -77,11 +73,6 @@ public abstract class BasicPartition<I extends WritableComparable,
   }
 
   @Override
-  public PartitionContext getPartitionContext() {
-    return partitionContext;
-  }
-
-  @Override
   public void progress() {
     if (progressable != null) {
       progressable.progress();
@@ -101,6 +92,5 @@ public abstract class BasicPartition<I extends WritableComparable,
   @Override
   public void readFields(DataInput input) throws IOException {
     id = input.readInt();
-    partitionContext = conf.createPartitionContext();
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java b/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
index c4669d3..6eaa6d7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
@@ -42,12 +42,11 @@ import java.util.concurrent.ConcurrentMap;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class ByteArrayPartition<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends BasicPartition<I, V, E, M>
-    implements ReusesObjectsPartition<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends BasicPartition<I, V, E>
+    implements ReusesObjectsPartition<I, V, E> {
   /**
    * Vertex map for this range (keyed by index).  Note that the byte[] is a
    * serialized vertex with the first four bytes as the length of the vertex
@@ -55,7 +54,7 @@ public class ByteArrayPartition<I extends WritableComparable,
    */
   private ConcurrentMap<I, byte[]> vertexMap;
   /** Representative vertex */
-  private Vertex<I, V, E, M> representativeVertex;
+  private Vertex<I, V, E> representativeVertex;
   /** Use unsafe serialization */
   private boolean useUnsafeSerialization;
 
@@ -78,7 +77,7 @@ public class ByteArrayPartition<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, M> getVertex(I vertexIndex) {
+  public Vertex<I, V, E> getVertex(I vertexIndex) {
     byte[] vertexData = vertexMap.get(vertexIndex);
     if (vertexData == null) {
       return null;
@@ -89,7 +88,7 @@ public class ByteArrayPartition<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, M> putVertex(Vertex<I, V, E, M> vertex) {
+  public Vertex<I, V, E> putVertex(Vertex<I, V, E> vertex) {
     byte[] vertexData =
         WritableUtils.writeVertexToByteArray(
             vertex, useUnsafeSerialization, getConf());
@@ -104,7 +103,7 @@ public class ByteArrayPartition<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, M> removeVertex(I vertexIndex) {
+  public Vertex<I, V, E> removeVertex(I vertexIndex) {
     byte[] vertexBytes = vertexMap.remove(vertexIndex);
     if (vertexBytes == null) {
       return null;
@@ -115,15 +114,15 @@ public class ByteArrayPartition<I extends WritableComparable,
   }
 
   @Override
-  public void addPartition(Partition<I, V, E, M> partition) {
+  public void addPartition(Partition<I, V, E> partition) {
     // Only work with other ByteArrayPartition instances
     if (!(partition instanceof ByteArrayPartition)) {
       throw new IllegalStateException("addPartition: Cannot add partition " +
           "of type " + partition.getClass());
     }
 
-    ByteArrayPartition<I, V, E, M> byteArrayPartition =
-        (ByteArrayPartition<I, V, E, M>) partition;
+    ByteArrayPartition<I, V, E> byteArrayPartition =
+        (ByteArrayPartition<I, V, E>) partition;
     for (Map.Entry<I, byte[]> entry :
         byteArrayPartition.vertexMap.entrySet()) {
       vertexMap.put(entry.getKey(), entry.getValue());
@@ -147,7 +146,7 @@ public class ByteArrayPartition<I extends WritableComparable,
   }
 
   @Override
-  public void saveVertex(Vertex<I, V, E, M> vertex) {
+  public void saveVertex(Vertex<I, V, E> vertex) {
     // Reuse the old buffer whenever possible
     byte[] oldVertexData = vertexMap.get(vertex.getId());
     if (oldVertexData != null) {
@@ -211,7 +210,7 @@ public class ByteArrayPartition<I extends WritableComparable,
   }
 
   @Override
-  public Iterator<Vertex<I, V, E, M>> iterator() {
+  public Iterator<Vertex<I, V, E>> iterator() {
     return new RepresentativeVertexIterator();
   }
 
@@ -220,7 +219,7 @@ public class ByteArrayPartition<I extends WritableComparable,
    * the same representative vertex object.
    */
   private class RepresentativeVertexIterator implements
-      Iterator<Vertex<I, V, E, M>> {
+      Iterator<Vertex<I, V, E>> {
     /** Iterator to the vertex values */
     private Iterator<byte[]> vertexDataIterator =
         vertexMap.values().iterator();
@@ -231,7 +230,7 @@ public class ByteArrayPartition<I extends WritableComparable,
     }
 
     @Override
-    public Vertex<I, V, E, M> next() {
+    public Vertex<I, V, E> next() {
       WritableUtils.reinitializeVertexFromByteArray(
           vertexDataIterator.next(), representativeVertex,
           useUnsafeSerialization, getConf());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/DefaultPartitionContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/DefaultPartitionContext.java b/giraph-core/src/main/java/org/apache/giraph/partition/DefaultPartitionContext.java
deleted file mode 100644
index c22c802..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/partition/DefaultPartitionContext.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.partition;
-
-import org.apache.giraph.worker.WorkerContext;
-
-/**
- * Empty implementation of {@link PartitionContext}
- */
-public class DefaultPartitionContext implements PartitionContext {
-  @Override
-  public void preSuperstep(WorkerContext workerContext) {
-  }
-
-  @Override
-  public void postSuperstep(WorkerContext workerContext) {
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java b/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
index a4739f1..dadce0f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
@@ -69,12 +69,11 @@ import static org.apache.giraph.conf.GiraphConstants.PARTITIONS_DIRECTORY;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class DiskBackedPartitionStore<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends PartitionStore<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends PartitionStore<I, V, E> {
   /** Class logger. */
   private static final Logger LOG =
       Logger.getLogger(DiskBackedPartitionStore.class);
@@ -92,9 +91,9 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
   /** Partitions' states store */
   private final Map<Integer, State> states = Maps.newHashMap();
   /** Current active partitions, which have not been put back yet */
-  private final Map<Integer, Partition<I, V, E, M>> active = Maps.newHashMap();
+  private final Map<Integer, Partition<I, V, E>> active = Maps.newHashMap();
   /** Inactive partitions to re-activate or spill to disk to make space */
-  private final Map<Integer, Partition<I, V, E, M>> inactive =
+  private final Map<Integer, Partition<I, V, E>> inactive =
       Maps.newLinkedHashMap();
   /** Ids of partitions stored on disk and number of vertices contained */
   private final Map<Integer, Integer> onDisk = Maps.newHashMap();
@@ -110,7 +109,8 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
   /** Executors for users requests. Uses caller threads */
   private final ExecutorService pool = new DirectExecutorService();
   /** Giraph configuration */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private final
+  ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Mapper context */
   private final Context context;
   /** Base path where the partition files are written to */
@@ -129,7 +129,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @param context Context
    */
   public DiskBackedPartitionStore(
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf,
       Mapper<?, ?, ?, ?>.Context context) {
     this.conf = conf;
     this.context = context;
@@ -222,7 +222,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
   }
 
   @Override
-  public Partition<I, V, E, M> getPartition(Integer id) {
+  public Partition<I, V, E> getPartition(Integer id) {
     try {
       return pool.submit(new GetPartition(id)).get();
     } catch (InterruptedException e) {
@@ -235,7 +235,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
   }
 
   @Override
-  public void putPartition(Partition<I, V, E, M> partition) {
+  public void putPartition(Partition<I, V, E> partition) {
     Integer id = partition.getId();
     try {
       pool.submit(new PutPartition(id, partition)).get();
@@ -262,8 +262,8 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
   }
 
   @Override
-  public Partition<I, V, E, M> removePartition(Integer id) {
-    Partition<I, V, E, M> partition = getPartition(id);
+  public Partition<I, V, E> removePartition(Integer id) {
+    Partition<I, V, E> partition = getPartition(id);
     // we put it back, so the partition can turn INACTIVE and be deleted.
     putPartition(partition);
     deletePartition(id);
@@ -271,7 +271,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
   }
 
   @Override
-  public void addPartition(Partition<I, V, E, M> partition) {
+  public void addPartition(Partition<I, V, E> partition) {
     Integer id = partition.getId();
     try {
       pool.submit(new AddPartition(partition.getId(), partition)).get();
@@ -307,11 +307,11 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
     StringBuilder sb = new StringBuilder();
     sb.append(partitionIds.toString());
     sb.append("\nActive\n");
-    for (Entry<Integer, Partition<I, V, E, M>> e : active.entrySet()) {
+    for (Entry<Integer, Partition<I, V, E>> e : active.entrySet()) {
       sb.append(e.getKey() + ":" + e.getValue() + "\n");
     }
     sb.append("Inactive\n");
-    for (Entry<Integer, Partition<I, V, E, M>> e : inactive.entrySet()) {
+    for (Entry<Integer, Partition<I, V, E>> e : inactive.entrySet()) {
       sb.append(e.getKey() + ":" + e.getValue() + "\n");
     }
     sb.append("OnDisk\n");
@@ -368,7 +368,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @param vertex The vertex to serialize
    * @throws IOException
    */
-  private void writeVertexData(DataOutput output, Vertex<I, V, E, M> vertex)
+  private void writeVertexData(DataOutput output, Vertex<I, V, E> vertex)
     throws IOException {
     vertex.getId().write(output);
     vertex.getValue().write(output);
@@ -383,7 +383,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
-  private void writeOutEdges(DataOutput output, Vertex<I, V, E, M> vertex)
+  private void writeOutEdges(DataOutput output, Vertex<I, V, E> vertex)
     throws IOException {
     vertex.getId().write(output);
     ((OutEdges<I, E>) vertex.getEdges()).write(output);
@@ -396,7 +396,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @param vertex The vertex to initialize
    * @throws IOException
    */
-  private void readVertexData(DataInput in, Vertex<I, V, E, M> vertex)
+  private void readVertexData(DataInput in, Vertex<I, V, E> vertex)
     throws IOException {
     I id = conf.createVertexId();
     id.readFields(in);
@@ -419,11 +419,11 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
-  private void readOutEdges(DataInput in, Partition<I, V, E, M> partition)
+  private void readOutEdges(DataInput in, Partition<I, V, E> partition)
     throws IOException {
     I id = conf.createVertexId();
     id.readFields(in);
-    Vertex<I, V, E, M> v = partition.getVertex(id);
+    Vertex<I, V, E> v = partition.getVertex(id);
     ((OutEdges<I, E>) v.getEdges()).readFields(in);
   }
 
@@ -437,9 +437,9 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @return The partition
    * @throws IOException
    */
-  private Partition<I, V, E, M> loadPartition(Integer id, int numVertices)
+  private Partition<I, V, E> loadPartition(Integer id, int numVertices)
     throws IOException {
-    Partition<I, V, E, M> partition =
+    Partition<I, V, E> partition =
         conf.createPartition(id, context);
     File file = new File(getVerticesPath(id));
     if (LOG.isDebugEnabled()) {
@@ -451,7 +451,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
       inputStream = new DataInputStream(
           new BufferedInputStream(new FileInputStream(file)));
       for (int i = 0; i < numVertices; ++i) {
-        Vertex<I, V , E, M> vertex = conf.createVertex();
+        Vertex<I, V , E> vertex = conf.createVertex();
         readVertexData(inputStream, vertex);
         partition.putVertex(vertex);
       }
@@ -493,7 +493,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @param partition The partition to offload
    * @throws IOException
    */
-  private void offloadPartition(Partition<I, V, E, M> partition)
+  private void offloadPartition(Partition<I, V, E> partition)
     throws IOException {
     File file = new File(getVerticesPath(partition.getId()));
     file.getParentFile().mkdirs();
@@ -506,7 +506,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
     try {
       outputStream = new DataOutputStream(
           new BufferedOutputStream(new FileOutputStream(file)));
-      for (Vertex<I, V, E, M> vertex : partition) {
+      for (Vertex<I, V, E> vertex : partition) {
         writeVertexData(outputStream, vertex);
       }
     } finally {
@@ -529,7 +529,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
       try {
         outputStream = new DataOutputStream(
             new BufferedOutputStream(new FileOutputStream(file)));
-        for (Vertex<I, V, E, M> vertex : partition) {
+        for (Vertex<I, V, E> vertex : partition) {
           writeOutEdges(outputStream, vertex);
         }
       } finally {
@@ -547,7 +547,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @param partition The partition
    * @throws IOException
    */
-  private void addToOOCPartition(Partition<I, V, E, M> partition)
+  private void addToOOCPartition(Partition<I, V, E> partition)
     throws IOException {
     Integer id = partition.getId();
     Integer count = onDisk.get(id);
@@ -557,7 +557,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
     try {
       outputStream = new DataOutputStream(
           new BufferedOutputStream(new FileOutputStream(file, true)));
-      for (Vertex<I, V, E, M> vertex : partition) {
+      for (Vertex<I, V, E> vertex : partition) {
         writeVertexData(outputStream, vertex);
       }
     } finally {
@@ -570,7 +570,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
     try {
       outputStream = new DataOutputStream(
           new BufferedOutputStream(new FileOutputStream(file, true)));
-      for (Vertex<I, V, E, M> vertex : partition) {
+      for (Vertex<I, V, E> vertex : partition) {
         writeOutEdges(outputStream, vertex);
       }
     } finally {
@@ -627,7 +627,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
   /**
    * Task that gets a partition from the store
    */
-  private class GetPartition implements Callable<Partition<I, V, E, M>> {
+  private class GetPartition implements Callable<Partition<I, V, E>> {
     /** Partition id */
     private Integer id;
 
@@ -645,17 +645,17 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
      *
      * @return The last recently used entry.
      */
-    private Entry<Integer, Partition<I, V, E, M>> getLRUEntry() {
-      Iterator<Entry<Integer, Partition<I, V, E, M>>> i =
+    private Entry<Integer, Partition<I, V, E>> getLRUEntry() {
+      Iterator<Entry<Integer, Partition<I, V, E>>> i =
           inactive.entrySet().iterator();
-      Entry<Integer, Partition<I, V, E, M>> lruEntry = i.next();
+      Entry<Integer, Partition<I, V, E>> lruEntry = i.next();
       i.remove();
       return lruEntry;
     }
 
     @Override
-    public Partition<I, V, E, M> call() throws Exception {
-      Partition<I, V, E, M> partition = null;
+    public Partition<I, V, E> call() throws Exception {
+      Partition<I, V, E> partition = null;
 
       while (partition == null) {
         wLock.lock();
@@ -663,7 +663,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
           State pState = states.get(id);
           switch (pState) {
           case ONDISK:
-            Entry<Integer, Partition<I, V, E, M>> lru = null;
+            Entry<Integer, Partition<I, V, E>> lru = null;
             states.put(id, State.LOADING);
             int numVertices = onDisk.remove(id);
             /*
@@ -747,7 +747,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
      * @param id The partition id
      * @param partition The partition
      */
-    public PutPartition(Integer id, Partition<I, V, E, M> partition) {
+    public PutPartition(Integer id, Partition<I, V, E> partition) {
       this.id = id;
     }
 
@@ -775,7 +775,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
     /** Partition id */
     private Integer id;
     /** Partition */
-    private Partition<I, V, E, M> partition;
+    private Partition<I, V, E> partition;
 
     /**
      * Constructor
@@ -783,7 +783,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
      * @param id The partition id
      * @param partition The partition
      */
-    public AddPartition(Integer id, Partition<I, V, E, M> partition) {
+    public AddPartition(Integer id, Partition<I, V, E> partition) {
       this.id = id;
       this.partition = partition;
     }
@@ -794,7 +794,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
       wLock.lock();
       try {
         if (partitionIds.contains(id)) {
-          Partition<I, V, E, M> existing = null;
+          Partition<I, V, E> existing = null;
           boolean isOOC = false;
           boolean done  = false;
           while (!done) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/GraphPartitionerFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/GraphPartitionerFactory.java b/giraph-core/src/main/java/org/apache/giraph/partition/GraphPartitionerFactory.java
index e2e04dd..4200d79 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/GraphPartitionerFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/GraphPartitionerFactory.java
@@ -28,11 +28,10 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public interface GraphPartitionerFactory<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> extends
+    V extends Writable, E extends Writable> extends
     ImmutableClassesGiraphConfigurable {
   /**
    * Create the {@link MasterGraphPartitioner} used by the master.
@@ -40,7 +39,7 @@ public interface GraphPartitionerFactory<I extends WritableComparable,
    *
    * @return Instantiated master graph partitioner
    */
-  MasterGraphPartitioner<I, V, E, M> createMasterGraphPartitioner();
+  MasterGraphPartitioner<I, V, E> createMasterGraphPartitioner();
 
   /**
    * Create the {@link WorkerGraphPartitioner} used by the worker.
@@ -48,5 +47,5 @@ public interface GraphPartitionerFactory<I extends WritableComparable,
    *
    * @return Instantiated worker graph partitioner
    */
-  WorkerGraphPartitioner<I, V, E, M> createWorkerGraphPartitioner();
+  WorkerGraphPartitioner<I, V, E> createWorkerGraphPartitioner();
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
index 5faf367..240687e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/HashMasterPartitioner.java
@@ -35,12 +35,11 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public class HashMasterPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    MasterGraphPartitioner<I, V, E, M> {
+    V extends Writable, E extends Writable> implements
+    MasterGraphPartitioner<I, V, E> {
   /** Class logger */
   private static Logger LOG = Logger.getLogger(HashMasterPartitioner.class);
   /** Provided configuration */

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/HashPartitionerFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/HashPartitionerFactory.java b/giraph-core/src/main/java/org/apache/giraph/partition/HashPartitionerFactory.java
index f7343a1..7cc5651 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/HashPartitionerFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/HashPartitionerFactory.java
@@ -29,23 +29,22 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public class HashPartitionerFactory<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements GraphPartitionerFactory<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    implements GraphPartitionerFactory<I, V, E> {
   /** Saved configuration */
   private ImmutableClassesGiraphConfiguration conf;
 
   @Override
-  public MasterGraphPartitioner<I, V, E, M> createMasterGraphPartitioner() {
-    return new HashMasterPartitioner<I, V, E, M>(getConf());
+  public MasterGraphPartitioner<I, V, E> createMasterGraphPartitioner() {
+    return new HashMasterPartitioner<I, V, E>(getConf());
   }
 
   @Override
-  public WorkerGraphPartitioner<I, V, E, M> createWorkerGraphPartitioner() {
-    return new HashWorkerPartitioner<I, V, E, M>();
+  public WorkerGraphPartitioner<I, V, E> createWorkerGraphPartitioner() {
+    return new HashWorkerPartitioner<I, V, E>();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/HashRangePartitionerFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/HashRangePartitionerFactory.java b/giraph-core/src/main/java/org/apache/giraph/partition/HashRangePartitionerFactory.java
index 227e234..1eeece7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/HashRangePartitionerFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/HashRangePartitionerFactory.java
@@ -29,23 +29,22 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public class HashRangePartitionerFactory<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements GraphPartitionerFactory<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    implements GraphPartitionerFactory<I, V, E> {
   /** Saved configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
 
   @Override
-  public MasterGraphPartitioner<I, V, E, M> createMasterGraphPartitioner() {
-    return new HashMasterPartitioner<I, V, E, M>(getConf());
+  public MasterGraphPartitioner<I, V, E> createMasterGraphPartitioner() {
+    return new HashMasterPartitioner<I, V, E>(getConf());
   }
 
   @Override
-  public WorkerGraphPartitioner<I, V, E, M> createWorkerGraphPartitioner() {
-    return new HashRangeWorkerPartitioner<I, V, E, M>();
+  public WorkerGraphPartitioner<I, V, E> createWorkerGraphPartitioner() {
+    return new HashRangeWorkerPartitioner<I, V, E>();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/HashRangeWorkerPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/HashRangeWorkerPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/HashRangeWorkerPartitioner.java
index a6e764d..81c3d7d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/HashRangeWorkerPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/HashRangeWorkerPartitioner.java
@@ -29,12 +29,11 @@ import com.google.common.primitives.UnsignedInts;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public class HashRangeWorkerPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends HashWorkerPartitioner<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends HashWorkerPartitioner<I, V, E> {
   /** A transformed hashCode() must be strictly smaller than this. */
   private static final long HASH_LIMIT = 2L * Integer.MAX_VALUE + 2L;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
index 599ea0c..d833895 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/HashWorkerPartitioner.java
@@ -32,12 +32,11 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public class HashWorkerPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements WorkerGraphPartitioner<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    implements WorkerGraphPartitioner<I, V, E> {
   /**
    * Mapping of the vertex ids to {@link PartitionOwner}.
    */
@@ -58,7 +57,7 @@ public class HashWorkerPartitioner<I extends WritableComparable,
   @Override
   public Collection<PartitionStats> finalizePartitionStats(
       Collection<PartitionStats> workerPartitionStats,
-      PartitionStore<I, V, E, M> partitionStore) {
+      PartitionStore<I, V, E> partitionStore) {
     // No modification necessary
     return workerPartitionStats;
   }
@@ -67,7 +66,7 @@ public class HashWorkerPartitioner<I extends WritableComparable,
   public PartitionExchange updatePartitionOwners(
       WorkerInfo myWorkerInfo,
       Collection<? extends PartitionOwner> masterSetPartitionOwners,
-      PartitionStore<I, V, E, M> partitionStore) {
+      PartitionStore<I, V, E> partitionStore) {
     return PartitionBalancer.updatePartitionOwners(partitionOwnerList,
         myWorkerInfo, masterSetPartitionOwners, partitionStore);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
index 130ee07..50c750a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/MasterGraphPartitioner.java
@@ -31,11 +31,10 @@ import org.apache.giraph.worker.WorkerInfo;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public interface MasterGraphPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /**
    * Set some initial partition owners for the graph. Guaranteed to be called
    * prior to the graph being loaded (initial or restart).

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java b/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
index 1ca0b61..aebd343 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
@@ -32,13 +32,12 @@ import org.apache.hadoop.util.Progressable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface Partition<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends Writable, ImmutableClassesGiraphConfigurable<I, V, E, M>,
-    Iterable<Vertex<I, V, E, M>> {
+    V extends Writable, E extends Writable>
+    extends Writable, ImmutableClassesGiraphConfigurable<I, V, E>,
+    Iterable<Vertex<I, V, E>> {
   /**
    * Initialize the partition.  Guaranteed to be called before used.
    *
@@ -53,7 +52,7 @@ public interface Partition<I extends WritableComparable,
    * @param vertexIndex Vertex index to search for
    * @return Vertex if it exists, null otherwise
    */
-  Vertex<I, V, E, M> getVertex(I vertexIndex);
+  Vertex<I, V, E> getVertex(I vertexIndex);
 
   /**
    * Put a vertex into the Partition
@@ -61,7 +60,7 @@ public interface Partition<I extends WritableComparable,
    * @param vertex Vertex to put in the Partition
    * @return old vertex value (i.e. null if none existed prior)
    */
-  Vertex<I, V, E, M> putVertex(Vertex<I, V, E, M> vertex);
+  Vertex<I, V, E> putVertex(Vertex<I, V, E> vertex);
 
   /**
    * Remove a vertex from the Partition
@@ -69,14 +68,14 @@ public interface Partition<I extends WritableComparable,
    * @param vertexIndex Vertex index to remove
    * @return The removed vertex.
    */
-  Vertex<I, V, E, M> removeVertex(I vertexIndex);
+  Vertex<I, V, E> removeVertex(I vertexIndex);
 
   /**
    * Add a partition's vertices
    *
    * @param partition Partition to add
    */
-  void addPartition(Partition<I, V, E, M> partition);
+  void addPartition(Partition<I, V, E> partition);
 
   /**
    * Get the number of vertices in this partition
@@ -123,12 +122,5 @@ public interface Partition<I extends WritableComparable,
    *
    * @param vertex Vertex to save
    */
-  void saveVertex(Vertex<I, V, E, M> vertex);
-
-  /**
-   * Get partition context
-   *
-   * @return Partition context
-   */
-  PartitionContext getPartitionContext();
+  void saveVertex(Vertex<I, V, E> vertex);
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/PartitionContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionContext.java b/giraph-core/src/main/java/org/apache/giraph/partition/PartitionContext.java
deleted file mode 100644
index 412f6e3..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionContext.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.partition;
-
-import org.apache.giraph.worker.WorkerContext;
-
-/**
- * PartitionContext allows for the execution of user code
- * on a per-partition basis. There's one PartitionContext per partition.
- */
-public interface PartitionContext {
-  /**
-   * Execute user code.
-   * This method is executed once for each partition before computation for
-   * that partition starts.
-   *
-   * @param workerContext Worker context
-   */
-  void preSuperstep(WorkerContext workerContext);
-
-  /**
-   * Execute user code.
-   * This method is executed once on for each partition after computation in
-   * current superstep for that partition ends.
-   *
-   * @param workerContext Worker context
-   */
-  void postSuperstep(WorkerContext workerContext);
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/PartitionStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionStore.java b/giraph-core/src/main/java/org/apache/giraph/partition/PartitionStore.java
index 4206ce3..763397e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/PartitionStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/PartitionStore.java
@@ -27,10 +27,9 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public abstract class PartitionStore<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
 
   /**
    * Add a new partition to the store or just the vertices from the partition
@@ -38,16 +37,16 @@ public abstract class PartitionStore<I extends WritableComparable,
    *
    * @param partition Partition to add
    */
-  public abstract void addPartition(Partition<I, V, E, M> partition);
+  public abstract void addPartition(Partition<I, V, E> partition);
 
   /**
    * Get a partition. Note: user has to put back it to the store through
-   * {@link #putPartition(Integer, Partition)} after use.
+   * {@link #putPartition(Partition)} after use.
    *
    * @param partitionId Partition id
    * @return The requested partition
    */
-  public abstract Partition<I, V, E, M> getPartition(Integer partitionId);
+  public abstract Partition<I, V, E> getPartition(Integer partitionId);
 
   /**
    * Put a partition back to the store. Use this method to be put a partition
@@ -55,7 +54,7 @@ public abstract class PartitionStore<I extends WritableComparable,
    *
    * @param partition Partition
    */
-  public abstract void putPartition(Partition<I, V, E, M> partition);
+  public abstract void putPartition(Partition<I, V, E> partition);
 
   /**
    * Remove a partition and return it.
@@ -63,7 +62,7 @@ public abstract class PartitionStore<I extends WritableComparable,
    * @param partitionId Partition id
    * @return The removed partition
    */
-  public abstract Partition<I, V, E, M> removePartition(Integer partitionId);
+  public abstract Partition<I, V, E> removePartition(Integer partitionId);
 
   /**
    * Just delete a partition

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/RangeMasterPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/RangeMasterPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/RangeMasterPartitioner.java
index f9b0329..3911a95 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/RangeMasterPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/RangeMasterPartitioner.java
@@ -30,12 +30,11 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public abstract class RangeMasterPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    MasterGraphPartitioner<I, V, E, M> {
+    V extends Writable, E extends Writable> implements
+    MasterGraphPartitioner<I, V, E> {
   @Override
   public PartitionStats createPartitionStats() {
     return new RangePartitionStats<I>();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/RangePartitionerFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/RangePartitionerFactory.java b/giraph-core/src/main/java/org/apache/giraph/partition/RangePartitionerFactory.java
index 29f7898..2ec4d4a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/RangePartitionerFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/RangePartitionerFactory.java
@@ -34,10 +34,9 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public abstract class RangePartitionerFactory<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements GraphPartitionerFactory<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    implements GraphPartitionerFactory<I, V, E> {
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/RangeSplitHint.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/RangeSplitHint.java b/giraph-core/src/main/java/org/apache/giraph/partition/RangeSplitHint.java
index 9634c33..4317944 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/RangeSplitHint.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/RangeSplitHint.java
@@ -43,7 +43,7 @@ public class RangeSplitHint<I extends WritableComparable>
   /** Number of vertices in this range after the split */
   private long postSplitVertexCount;
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, ?, ?, ?> conf;
+  private ImmutableClassesGiraphConfiguration<I, ?, ?> conf;
 
   @Override
   public void readFields(DataInput input) throws IOException {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/RangeWorkerPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/RangeWorkerPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/RangeWorkerPartitioner.java
index 5a494a5..cbcd753 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/RangeWorkerPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/RangeWorkerPartitioner.java
@@ -39,12 +39,11 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 @SuppressWarnings("rawtypes")
 public abstract class RangeWorkerPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    WorkerGraphPartitioner<I, V, E, M> {
+    V extends Writable, E extends Writable> implements
+    WorkerGraphPartitioner<I, V, E> {
   /** Mapping of the vertex ids to the {@link PartitionOwner} */
   protected NavigableMap<I, RangePartitionOwner<I>> vertexRangeMap =
       new TreeMap<I, RangePartitionOwner<I>>();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/ReusesObjectsPartition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/ReusesObjectsPartition.java b/giraph-core/src/main/java/org/apache/giraph/partition/ReusesObjectsPartition.java
index 4153d55..405bc5c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/ReusesObjectsPartition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/ReusesObjectsPartition.java
@@ -29,8 +29,6 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message value
  */
 public interface ReusesObjectsPartition<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends Partition<I, V, E, M> { }
+    V extends Writable, E extends Writable> extends Partition<I, V, E> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/SimpleIntRangePartitionerFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleIntRangePartitionerFactory.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleIntRangePartitionerFactory.java
index 9ac2e11..7aee84c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleIntRangePartitionerFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleIntRangePartitionerFactory.java
@@ -30,26 +30,25 @@ import org.apache.hadoop.io.Writable;
  *
  * @param <V> Vertex value type
  * @param <E> Edge value type
- * @param <M> Message data type
  */
 public class SimpleIntRangePartitionerFactory<V extends Writable,
-    E extends Writable, M extends Writable>
-    implements GraphPartitionerFactory<IntWritable, V, E, M> {
+    E extends Writable>
+    implements GraphPartitionerFactory<IntWritable, V, E> {
   /** Configuration. */
   private ImmutableClassesGiraphConfiguration conf;
   /** Vertex key space size. */
   private long keySpaceSize;
 
   @Override
-  public MasterGraphPartitioner<IntWritable, V, E, M>
+  public MasterGraphPartitioner<IntWritable, V, E>
   createMasterGraphPartitioner() {
-    return new SimpleRangeMasterPartitioner<IntWritable, V, E, M>(conf);
+    return new SimpleRangeMasterPartitioner<IntWritable, V, E>(conf);
   }
 
   @Override
-  public WorkerGraphPartitioner<IntWritable, V, E, M>
+  public WorkerGraphPartitioner<IntWritable, V, E>
   createWorkerGraphPartitioner() {
-    return new SimpleRangeWorkerPartitioner<IntWritable, V, E, M>(
+    return new SimpleRangeWorkerPartitioner<IntWritable, V, E>(
         keySpaceSize) {
       @Override
       protected long vertexKeyFromId(IntWritable id) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/SimpleLongRangePartitionerFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleLongRangePartitionerFactory.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleLongRangePartitionerFactory.java
index 5772a7b..64efde9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleLongRangePartitionerFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleLongRangePartitionerFactory.java
@@ -30,26 +30,25 @@ import org.apache.hadoop.io.Writable;
  *
  * @param <V> Vertex value type
  * @param <E> Edge value type
- * @param <M> Message data type
  */
 public class SimpleLongRangePartitionerFactory<V extends Writable,
-    E extends Writable, M extends Writable>
-    implements GraphPartitionerFactory<LongWritable, V, E, M> {
+    E extends Writable>
+    implements GraphPartitionerFactory<LongWritable, V, E> {
   /** Configuration. */
   private ImmutableClassesGiraphConfiguration conf;
   /** Vertex key space size. */
   private long keySpaceSize;
 
   @Override
-  public MasterGraphPartitioner<LongWritable, V, E, M>
+  public MasterGraphPartitioner<LongWritable, V, E>
   createMasterGraphPartitioner() {
-    return new SimpleRangeMasterPartitioner<LongWritable, V, E, M>(conf);
+    return new SimpleRangeMasterPartitioner<LongWritable, V, E>(conf);
   }
 
   @Override
-  public WorkerGraphPartitioner<LongWritable, V, E, M>
+  public WorkerGraphPartitioner<LongWritable, V, E>
   createWorkerGraphPartitioner() {
-    return new SimpleRangeWorkerPartitioner<LongWritable, V, E, M>(
+    return new SimpleRangeWorkerPartitioner<LongWritable, V, E>(
         keySpaceSize) {
       @Override
       protected long vertexKeyFromId(LongWritable id) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartition.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartition.java
index d6a46bd..0c1b404 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartition.java
@@ -42,14 +42,13 @@ import static org.apache.giraph.conf.GiraphConstants.USE_OUT_OF_CORE_MESSAGES;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class SimplePartition<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends BasicPartition<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends BasicPartition<I, V, E> {
   /** Vertex map for this range (keyed by index) */
-  private ConcurrentMap<I, Vertex<I, V, E, M>> vertexMap;
+  private ConcurrentMap<I, Vertex<I, V, E>> vertexMap;
 
   /**
    * Constructor for reflection.
@@ -60,30 +59,30 @@ public class SimplePartition<I extends WritableComparable,
   public void initialize(int partitionId, Progressable progressable) {
     super.initialize(partitionId, progressable);
     if (USE_OUT_OF_CORE_MESSAGES.get(getConf())) {
-      vertexMap = new ConcurrentSkipListMap<I, Vertex<I, V, E, M>>();
+      vertexMap = new ConcurrentSkipListMap<I, Vertex<I, V, E>>();
     } else {
       vertexMap = Maps.newConcurrentMap();
     }
   }
 
   @Override
-  public Vertex<I, V, E, M> getVertex(I vertexIndex) {
+  public Vertex<I, V, E> getVertex(I vertexIndex) {
     return vertexMap.get(vertexIndex);
   }
 
   @Override
-  public Vertex<I, V, E, M> putVertex(Vertex<I, V, E, M> vertex) {
+  public Vertex<I, V, E> putVertex(Vertex<I, V, E> vertex) {
     return vertexMap.put(vertex.getId(), vertex);
   }
 
   @Override
-  public Vertex<I, V, E, M> removeVertex(I vertexIndex) {
+  public Vertex<I, V, E> removeVertex(I vertexIndex) {
     return vertexMap.remove(vertexIndex);
   }
 
   @Override
-  public void addPartition(Partition<I, V, E, M> partition) {
-    for (Vertex<I, V, E , M> vertex : partition) {
+  public void addPartition(Partition<I, V, E> partition) {
+    for (Vertex<I, V, E> vertex : partition) {
       vertexMap.put(vertex.getId(), vertex);
     }
   }
@@ -96,14 +95,14 @@ public class SimplePartition<I extends WritableComparable,
   @Override
   public long getEdgeCount() {
     long edges = 0;
-    for (Vertex<I, V, E, M> vertex : vertexMap.values()) {
+    for (Vertex<I, V, E> vertex : vertexMap.values()) {
       edges += vertex.getNumEdges();
     }
     return edges;
   }
 
   @Override
-  public void saveVertex(Vertex<I, V, E, M> vertex) {
+  public void saveVertex(Vertex<I, V, E> vertex) {
     // No-op, vertices are stored as Java objects in this partition
   }
 
@@ -116,14 +115,14 @@ public class SimplePartition<I extends WritableComparable,
   public void readFields(DataInput input) throws IOException {
     super.readFields(input);
     if (USE_OUT_OF_CORE_MESSAGES.get(getConf())) {
-      vertexMap = new ConcurrentSkipListMap<I, Vertex<I, V, E, M>>();
+      vertexMap = new ConcurrentSkipListMap<I, Vertex<I, V, E>>();
     } else {
       vertexMap = Maps.newConcurrentMap();
     }
     int vertices = input.readInt();
     for (int i = 0; i < vertices; ++i) {
       progress();
-      Vertex<I, V, E, M> vertex =
+      Vertex<I, V, E> vertex =
           WritableUtils.readVertexFromDataInput(input, getConf());
       if (vertexMap.put(vertex.getId(), vertex) != null) {
         throw new IllegalStateException(
@@ -137,14 +136,14 @@ public class SimplePartition<I extends WritableComparable,
   public void write(DataOutput output) throws IOException {
     super.write(output);
     output.writeInt(vertexMap.size());
-    for (Vertex<I, V, E, M> vertex : vertexMap.values()) {
+    for (Vertex<I, V, E> vertex : vertexMap.values()) {
       progress();
       WritableUtils.writeVertexToDataOutput(output, vertex, getConf());
     }
   }
 
   @Override
-  public Iterator<Vertex<I, V, E, M>> iterator() {
+  public Iterator<Vertex<I, V, E>> iterator() {
     return vertexMap.values().iterator();
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartitionStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartitionStore.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartitionStore.java
index 74cc3a7..ae17aac 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartitionStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimplePartitionStore.java
@@ -33,16 +33,15 @@ import java.util.concurrent.ConcurrentMap;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public class SimplePartitionStore<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends PartitionStore<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends PartitionStore<I, V, E> {
   /** Map of stored partitions. */
-  private final ConcurrentMap<Integer, Partition<I, V, E, M>> partitions =
+  private final ConcurrentMap<Integer, Partition<I, V, E>> partitions =
       Maps.newConcurrentMap();
   /** Configuration. */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Context used to report progress */
   private final Mapper<?, ?, ?, ?>.Context context;
 
@@ -53,15 +52,15 @@ public class SimplePartitionStore<I extends WritableComparable,
    * @param context Mapper context
    */
   public SimplePartitionStore(
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf,
       Mapper<?, ?, ?, ?>.Context context) {
     this.conf = conf;
     this.context = context;
   }
 
   @Override
-  public void addPartition(Partition<I, V, E, M> partition) {
-    Partition<I, V, E, M> oldPartition = partitions.get(partition.getId());
+  public void addPartition(Partition<I, V, E> partition) {
+    Partition<I, V, E> oldPartition = partitions.get(partition.getId());
     if (oldPartition == null) {
       oldPartition = partitions.putIfAbsent(partition.getId(), partition);
       if (oldPartition == null) {
@@ -72,12 +71,12 @@ public class SimplePartitionStore<I extends WritableComparable,
   }
 
   @Override
-  public Partition<I, V, E, M> getPartition(Integer partitionId) {
+  public Partition<I, V, E> getPartition(Integer partitionId) {
     return partitions.get(partitionId);
   }
 
   @Override
-  public Partition<I, V, E, M> removePartition(Integer partitionId) {
+  public Partition<I, V, E> removePartition(Integer partitionId) {
     return partitions.remove(partitionId);
   }
 
@@ -102,5 +101,5 @@ public class SimplePartitionStore<I extends WritableComparable,
   }
 
   @Override
-  public void putPartition(Partition<I, V, E, M> partition) { }
+  public void putPartition(Partition<I, V, E> partition) { }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeMasterPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeMasterPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeMasterPartitioner.java
index bf34ecd..37ce8c7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeMasterPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeMasterPartitioner.java
@@ -37,11 +37,10 @@ import java.util.List;
  * @param <I> Vertex id type
  * @param <V> Vertex value type
  * @param <E> Edge value type
- * @param <M> Message data type
  */
 public class SimpleRangeMasterPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    MasterGraphPartitioner<I, V, E, M> {
+    V extends Writable, E extends Writable> implements
+    MasterGraphPartitioner<I, V, E> {
   /** Class logger */
   private static Logger LOG = Logger.getLogger(HashMasterPartitioner.class);
   /** Provided configuration */

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeWorkerPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeWorkerPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeWorkerPartitioner.java
index f94c14b..ab2afd5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeWorkerPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/SimpleRangeWorkerPartitioner.java
@@ -35,12 +35,10 @@ import java.util.List;
  * @param <I> Vertex id type
  * @param <V> Vertex value type
  * @param <E> Edge value type
- * @param <M> Message data type
  */
 public abstract class SimpleRangeWorkerPartitioner<I extends
-    WritableComparable, V extends Writable, E extends Writable,
-    M extends Writable>
-    implements WorkerGraphPartitioner<I, V, E, M> {
+    WritableComparable, V extends Writable, E extends Writable>
+    implements WorkerGraphPartitioner<I, V, E> {
   /** List of {@link PartitionOwner}s for this worker. */
   private List<PartitionOwner> partitionOwnerList = Lists.newArrayList();
   /** Vertex keys space size. */
@@ -88,7 +86,7 @@ public abstract class SimpleRangeWorkerPartitioner<I extends
   @Override
   public Collection<PartitionStats> finalizePartitionStats(
       Collection<PartitionStats> workerPartitionStats,
-      PartitionStore<I, V, E, M> partitionStore) {
+      PartitionStore<I, V, E> partitionStore) {
     // No modification necessary
     return workerPartitionStats;
   }
@@ -97,7 +95,7 @@ public abstract class SimpleRangeWorkerPartitioner<I extends
   public PartitionExchange updatePartitionOwners(
       WorkerInfo myWorkerInfo,
       Collection<? extends PartitionOwner> masterSetPartitionOwners,
-      PartitionStore<I, V, E, M> partitionStore) {
+      PartitionStore<I, V, E> partitionStore) {
     return PartitionBalancer.updatePartitionOwners(partitionOwnerList,
         myWorkerInfo, masterSetPartitionOwners, partitionStore);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java b/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
index 5a78b1d..004ea81 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/WorkerGraphPartitioner.java
@@ -32,11 +32,10 @@ import java.util.Collection;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface WorkerGraphPartitioner<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /**
    * Instantiate the {@link PartitionOwner} implementation used to read the
    * master assignments.
@@ -67,7 +66,7 @@ public interface WorkerGraphPartitioner<I extends WritableComparable,
    */
   Collection<PartitionStats> finalizePartitionStats(
       Collection<PartitionStats> workerPartitionStats,
-      PartitionStore<I, V, E, M> partitionStore);
+      PartitionStore<I, V, E> partitionStore);
 
   /**
    * Get the partitions owners and update locally.  Returns the partitions
@@ -83,7 +82,7 @@ public interface WorkerGraphPartitioner<I extends WritableComparable,
   PartitionExchange updatePartitionOwners(
       WorkerInfo myWorkerInfo,
       Collection<? extends PartitionOwner> masterSetPartitionOwners,
-      PartitionStore<I, V, E, M> partitionStore);
+      PartitionStore<I, V, E> partitionStore);
 
   /**
    * Get a collection of the {@link PartitionOwner} objects.

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdData.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdData.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdData.java
index 2cfa661..9b3f165 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdData.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdData.java
@@ -38,7 +38,7 @@ public abstract class ByteArrayVertexIdData<I extends WritableComparable, T>
   /** Extended data output */
   private ExtendedDataOutput extendedDataOutput;
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, ?, ?, ?> configuration;
+  private ImmutableClassesGiraphConfiguration<I, ?, ?> configuration;
 
   /**
    * Create a new data object.
@@ -149,7 +149,7 @@ public abstract class ByteArrayVertexIdData<I extends WritableComparable, T>
   }
 
   @Override
-  public ImmutableClassesGiraphConfiguration getConf() {
+  public ImmutableClassesGiraphConfiguration<I, ?, ?> getConf() {
     return configuration;
   }
 


[02/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestMaxSuperstep.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestMaxSuperstep.java b/giraph-examples/src/test/java/org/apache/giraph/TestMaxSuperstep.java
index 0989ac5..75157a1 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestMaxSuperstep.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestMaxSuperstep.java
@@ -21,8 +21,11 @@ package org.apache.giraph;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.counters.GiraphHadoopCounter;
 import org.apache.giraph.counters.GiraphStats;
-import org.apache.giraph.examples.SimplePageRankVertex.SimplePageRankVertexInputFormat;
-import org.apache.giraph.examples.SimplePageRankVertex.SimplePageRankVertexOutputFormat;
+import org.apache.giraph.examples.SimplePageRankComputation.SimplePageRankVertexInputFormat;
+import org.apache.giraph.examples.SimplePageRankComputation.SimplePageRankVertexOutputFormat;
+
+
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.hadoop.io.DoubleWritable;
@@ -47,11 +50,12 @@ public class TestMaxSuperstep extends BspCase {
    * Simple test vertex class that will run forever (voteToHalt is never
    * called).
    */
-  public static class InfiniteLoopVertex extends Vertex<LongWritable,
-      DoubleWritable, FloatWritable, DoubleWritable> {
+  public static class InfiniteLoopComputation extends BasicComputation<
+      LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
     @Override
-    public void compute(Iterable<DoubleWritable> messages) throws IOException {
-      // Do nothing, run forever!
+    public void compute(
+        Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+        Iterable<DoubleWritable> messages) throws IOException {
     }
   }
 
@@ -67,7 +71,7 @@ public class TestMaxSuperstep extends BspCase {
   public void testMaxSuperstep()
           throws IOException, InterruptedException, ClassNotFoundException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(InfiniteLoopVertex.class);
+    conf.setComputationClass(InfiniteLoopComputation.class);
     conf.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestMutateGraph.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestMutateGraph.java b/giraph-examples/src/test/java/org/apache/giraph/TestMutateGraph.java
index da85fc4..91252f3 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestMutateGraph.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestMutateGraph.java
@@ -19,9 +19,9 @@
 package org.apache.giraph;
 
 import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.examples.SimpleMutateGraphVertex;
-import org.apache.giraph.examples.SimplePageRankVertex.SimplePageRankVertexInputFormat;
-import org.apache.giraph.examples.SimplePageRankVertex.SimplePageRankVertexOutputFormat;
+import org.apache.giraph.examples.SimpleMutateGraphComputation;
+import org.apache.giraph.examples.SimplePageRankComputation.SimplePageRankVertexInputFormat;
+import org.apache.giraph.examples.SimplePageRankComputation.SimplePageRankVertexOutputFormat;
 import org.apache.giraph.job.GiraphJob;
 import org.junit.Test;
 
@@ -48,11 +48,11 @@ public class TestMutateGraph extends BspCase {
   public void testMutateGraph()
           throws IOException, InterruptedException, ClassNotFoundException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimpleMutateGraphVertex.class);
+    conf.setComputationClass(SimpleMutateGraphComputation.class);
     conf.setVertexInputFormatClass(SimplePageRankVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
     conf.setWorkerContextClass(
-        SimpleMutateGraphVertex.SimpleMutateGraphVertexWorkerContext.class);
+        SimpleMutateGraphComputation.SimpleMutateGraphVertexWorkerContext.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf,
         getTempPath(getCallingMethodName()));
     assertTrue(job.run(true));

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java b/giraph-examples/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java
index 6da9c50..af16bb6 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestNotEnoughMapTasks.java
@@ -19,9 +19,9 @@
 package org.apache.giraph;
 
 import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.examples.SimpleCheckpointVertex;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexOutputFormat;
+import org.apache.giraph.examples.SimpleCheckpoint;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexOutputFormat;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
@@ -56,8 +56,8 @@ public class TestNotEnoughMapTasks extends BspCase {
     }
     Path outputPath = getTempPath(getCallingMethodName());
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(
-        SimpleCheckpointVertex.SimpleCheckpointComputation.class);
+    conf.setComputationClass(
+        SimpleCheckpoint.SimpleCheckpointComputation.class);
     conf.setVertexInputFormatClass(SimpleSuperstepVertexInputFormat.class);
     conf.setVertexOutputFormatClass(SimpleSuperstepVertexOutputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/TestPartitionContext.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestPartitionContext.java b/giraph-examples/src/test/java/org/apache/giraph/TestPartitionContext.java
deleted file mode 100644
index 4b042df..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/TestPartitionContext.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph;
-
-import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.examples.GeneratedVertexReader;
-import org.apache.giraph.examples.PartitionContextTestVertex;
-import org.apache.giraph.examples.SimplePageRankVertex;
-import org.apache.giraph.job.GiraphJob;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertTrue;
-
-public class TestPartitionContext extends BspCase {
-  public TestPartitionContext() {
-    super(TestPartitionContext.class.getName());
-  }
-
-  @Test
-  public void testPartitionContext() throws IOException,
-      ClassNotFoundException, InterruptedException {
-    if (runningInDistributedMode()) {
-      System.out.println(
-          "testComputeContext: Ignore this test in distributed mode.");
-      return;
-    }
-    GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(PartitionContextTestVertex.class);
-    conf.setVertexInputFormatClass(
-        SimplePageRankVertex.SimplePageRankVertexInputFormat.class);
-    conf.setWorkerContextClass(
-        PartitionContextTestVertex.TestPartitionContextWorkerContext.class);
-    conf.setPartitionContextClass(
-        PartitionContextTestVertex.TestPartitionContextPartitionContext.class);
-    GiraphJob job = prepareJob(getCallingMethodName(), conf);
-    // Use multithreading
-    job.getConfiguration().setNumComputeThreads(
-        PartitionContextTestVertex.NUM_COMPUTE_THREADS);
-    // Increase the number of vertices
-    job.getConfiguration().setInt(
-        GeneratedVertexReader.READER_VERTICES,
-        PartitionContextTestVertex.NUM_VERTICES);
-    // Increase the number of partitions
-    GiraphConstants.USER_PARTITION_COUNT.set(job.getConfiguration(),
-        PartitionContextTestVertex.NUM_PARTITIONS);
-    assertTrue(job.run(true));
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java b/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
index 53cdeab..6d22800 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/aggregators/TestAggregatorsHandling.java
@@ -23,9 +23,9 @@ import org.apache.giraph.comm.aggregators.AggregatorUtils;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.examples.AggregatorsTestVertex;
-import org.apache.giraph.examples.SimpleCheckpointVertex;
-import org.apache.giraph.examples.SimplePageRankVertex;
+import org.apache.giraph.examples.AggregatorsTestComputation;
+import org.apache.giraph.examples.SimpleCheckpoint;
+import org.apache.giraph.examples.SimplePageRankComputation;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.master.MasterAggregatorHandler;
 import org.apache.hadoop.fs.Path;
@@ -75,12 +75,12 @@ public class TestAggregatorsHandling extends BspCase {
   public void testAggregatorsHandling() throws IOException,
       ClassNotFoundException, InterruptedException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(AggregatorsTestVertex.class);
+    conf.setComputationClass(AggregatorsTestComputation.class);
     conf.setVertexInputFormatClass(
-        SimplePageRankVertex.SimplePageRankVertexInputFormat.class);
+        SimplePageRankComputation.SimplePageRankVertexInputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf);
     job.getConfiguration().setMasterComputeClass(
-        AggregatorsTestVertex.AggregatorsTestMasterCompute.class);
+        AggregatorsTestComputation.AggregatorsTestMasterCompute.class);
     // test with aggregators split in a few requests
     job.getConfiguration().setInt(
         AggregatorUtils.MAX_BYTES_PER_AGGREGATOR_REQUEST, 50);
@@ -155,11 +155,11 @@ public class TestAggregatorsHandling extends BspCase {
     Path checkpointsDir = getTempPath("checkPointsForTesting");
     Path outputPath = getTempPath(getCallingMethodName());
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(AggregatorsTestVertex.class);
+    conf.setComputationClass(AggregatorsTestComputation.class);
     conf.setMasterComputeClass(
-        AggregatorsTestVertex.AggregatorsTestMasterCompute.class);
+        AggregatorsTestComputation.AggregatorsTestMasterCompute.class);
     conf.setVertexInputFormatClass(
-        SimplePageRankVertex.SimplePageRankVertexInputFormat.class);
+        SimplePageRankComputation.SimplePageRankVertexInputFormat.class);
     GiraphJob job = prepareJob(getCallingMethodName(), conf, outputPath);
 
     GiraphConfiguration configuration = job.getConfiguration();
@@ -174,15 +174,15 @@ public class TestAggregatorsHandling extends BspCase {
         "superstep 4 with checkpoint path = " + checkpointsDir);
     outputPath = getTempPath(getCallingMethodName() + "Restarted");
     conf = new GiraphConfiguration();
-    conf.setVertexClass(AggregatorsTestVertex.class);
+    conf.setComputationClass(AggregatorsTestComputation.class);
     conf.setMasterComputeClass(
-        AggregatorsTestVertex.AggregatorsTestMasterCompute.class);
+        AggregatorsTestComputation.AggregatorsTestMasterCompute.class);
     conf.setVertexInputFormatClass(
-        SimplePageRankVertex.SimplePageRankVertexInputFormat.class);
+        SimplePageRankComputation.SimplePageRankVertexInputFormat.class);
     GiraphJob restartedJob = prepareJob(getCallingMethodName() + "Restarted",
         conf, outputPath);
     job.getConfiguration().setMasterComputeClass(
-        SimpleCheckpointVertex.SimpleCheckpointVertexMasterCompute.class);
+        SimpleCheckpoint.SimpleCheckpointVertexMasterCompute.class);
     GiraphConfiguration restartedJobConf = restartedJob.getConfiguration();
     GiraphConstants.CHECKPOINT_DIRECTORY.set(restartedJobConf,
         checkpointsDir.toString());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTest.java
new file mode 100644
index 0000000..7d326da
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTest.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.combiner.MinimumIntCombiner;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
+import org.apache.giraph.io.formats.IntIntNullTextInputFormat;
+import org.apache.giraph.utils.InternalVertexRunner;
+import org.junit.Test;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.SetMultimap;
+
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Tests for {@link ConnectedComponentsComputation}
+ */
+public class ConnectedComponentsComputationTest {
+
+    /**
+     * A local integration test on toy data
+     */
+    @Test
+    public void testToyData() throws Exception {
+
+        // a small graph with three components
+        String[] graph = new String[] {
+                "1 2 3",
+                "2 1 4 5",
+                "3 1 4",
+                "4 2 3 5 13",
+                "5 2 4 12 13",
+                "12 5 13",
+                "13 4 5 12",
+
+                "6 7 8",
+                "7 6 10 11",
+                "8 6 10",
+                "10 7 8 11",
+                "11 7 10",
+
+                "9" };
+
+        GiraphConfiguration conf = new GiraphConfiguration();
+        conf.setComputationClass(ConnectedComponentsComputation.class);
+        conf.setOutEdgesClass(ByteArrayEdges.class);
+        conf.setCombinerClass(MinimumIntCombiner.class);
+        conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
+        conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
+
+        // run internally
+        Iterable<String> results = InternalVertexRunner.run(conf, graph);
+
+        SetMultimap<Integer,Integer> components = parseResults(results);
+
+        Set<Integer> componentIDs = components.keySet();
+        assertEquals(3, componentIDs.size());
+        assertTrue(componentIDs.contains(1));
+        assertTrue(componentIDs.contains(6));
+        assertTrue(componentIDs.contains(9));
+
+        Set<Integer> componentOne = components.get(1);
+        assertEquals(7, componentOne.size());
+        assertTrue(componentOne.contains(1));
+        assertTrue(componentOne.contains(2));
+        assertTrue(componentOne.contains(3));
+        assertTrue(componentOne.contains(4));
+        assertTrue(componentOne.contains(5));
+        assertTrue(componentOne.contains(12));
+        assertTrue(componentOne.contains(13));
+
+        Set<Integer> componentTwo = components.get(6);
+        assertEquals(5, componentTwo.size());
+        assertTrue(componentTwo.contains(6));
+        assertTrue(componentTwo.contains(7));
+        assertTrue(componentTwo.contains(8));
+        assertTrue(componentTwo.contains(10));
+        assertTrue(componentTwo.contains(11));
+
+        Set<Integer> componentThree = components.get(9);
+        assertEquals(1, componentThree.size());
+        assertTrue(componentThree.contains(9));
+    }
+
+    private SetMultimap<Integer,Integer> parseResults(
+            Iterable<String> results) {
+        SetMultimap<Integer,Integer> components = HashMultimap.create();
+        for (String result : results) {
+            Iterable<String> parts = Splitter.on('\t').split(result);
+            int vertex = Integer.parseInt(Iterables.get(parts, 0));
+            int component = Integer.parseInt(Iterables.get(parts, 1));
+            components.put(component, vertex);
+        }
+        return components;
+    }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java
new file mode 100644
index 0000000..b94f8dc
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.combiner.MinimumIntCombiner;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.InternalVertexRunner;
+import org.apache.giraph.utils.TestGraph;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.junit.Test;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.SetMultimap;
+
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *  Tests for {@link ConnectedComponentsComputation}
+ */
+public class ConnectedComponentsComputationTestInMemory {
+  public static Entry<IntWritable, NullWritable>[] makeEdges(int... args){
+    Entry<IntWritable, NullWritable> result[] =
+      new Entry[args.length];
+    for (int i=0; i<args.length; i++){
+      result[i] = new SimpleEntry<IntWritable, NullWritable>(
+          new IntWritable(args[i]), NullWritable.get());
+    }
+    return result;
+  }
+  /**
+   * A local integration test on toy data
+   */
+  @Test
+  public void testToyData() throws Exception {
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setComputationClass(ConnectedComponentsComputation.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
+    conf.setCombinerClass(MinimumIntCombiner.class);
+
+    TestGraph<IntWritable, IntWritable, NullWritable> graph =
+      new TestGraph<IntWritable, IntWritable, NullWritable> (conf);
+    // a small graph with three components
+    graph.addVertex(new IntWritable(1), new IntWritable(1), makeEdges(2, 3))
+      .addVertex(new IntWritable(2), new IntWritable(2), makeEdges(1, 4, 5))
+      .addVertex(new IntWritable(3), new IntWritable(3), makeEdges(1, 4))
+      .addVertex(new IntWritable(4), new IntWritable(4),
+          makeEdges(2, 3, 5, 13))
+      .addVertex(new IntWritable(5), new IntWritable(5),
+          makeEdges(2, 4, 12, 13))
+      .addVertex(new IntWritable(12), new IntWritable(12), makeEdges(5, 13))
+      .addVertex(new IntWritable(13), new IntWritable(13), makeEdges(4, 5, 12))
+      .addVertex(new IntWritable(6), new IntWritable(6), makeEdges(7, 8))
+      .addVertex(new IntWritable(7), new IntWritable(7), makeEdges(6, 10, 11))
+      .addVertex(new IntWritable(8), new IntWritable(8), makeEdges(6, 10))
+      .addVertex(new IntWritable(10), new IntWritable(10), makeEdges(7, 8, 11))
+      .addVertex(new IntWritable(11), new IntWritable(11), makeEdges(7, 10))
+      .addVertex(new IntWritable(9), new IntWritable(9));
+
+    // run internally
+    TestGraph<IntWritable, IntWritable, NullWritable> results =
+      InternalVertexRunner.run(conf, graph);
+
+    SetMultimap<Integer,Integer> components = parseResults(results);
+
+    Set<Integer> componentIDs = components.keySet();
+    assertEquals(3, componentIDs.size());
+    assertTrue(componentIDs.contains(1));
+    assertTrue(componentIDs.contains(6));
+    assertTrue(componentIDs.contains(9));
+
+    Set<Integer> componentOne = components.get(1);
+    assertEquals(7, componentOne.size());
+    assertTrue(componentOne.contains(1));
+    assertTrue(componentOne.contains(2));
+    assertTrue(componentOne.contains(3));
+    assertTrue(componentOne.contains(4));
+    assertTrue(componentOne.contains(5));
+    assertTrue(componentOne.contains(12));
+    assertTrue(componentOne.contains(13));
+
+    Set<Integer> componentTwo = components.get(6);
+    assertEquals(5, componentTwo.size());
+    assertTrue(componentTwo.contains(6));
+    assertTrue(componentTwo.contains(7));
+    assertTrue(componentTwo.contains(8));
+    assertTrue(componentTwo.contains(10));
+    assertTrue(componentTwo.contains(11));
+
+    Set<Integer> componentThree = components.get(9);
+    assertEquals(1, componentThree.size());
+    assertTrue(componentThree.contains(9));
+  }
+
+  private SetMultimap<Integer,Integer> parseResults(
+    TestGraph<IntWritable, IntWritable, NullWritable> results) {
+    SetMultimap<Integer,Integer> components = HashMultimap.create();
+    for (Vertex<IntWritable, IntWritable, NullWritable> vertex : results) {
+      int component = vertex.getValue().get();
+      components.put(component, vertex.getId().get());
+    }
+    return components;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
deleted file mode 100644
index 5d71259..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.combiner.MinimumIntCombiner;
-import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
-import org.apache.giraph.io.formats.IntIntNullTextInputFormat;
-import org.apache.giraph.utils.InternalVertexRunner;
-import org.junit.Test;
-
-import com.google.common.base.Splitter;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.SetMultimap;
-
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- *  Tests for {@link ConnectedComponentsVertex}
- */
-public class ConnectedComponentsVertexTest {
-
-    /**
-     * A local integration test on toy data
-     */
-    @Test
-    public void testToyData() throws Exception {
-
-        // a small graph with three components
-        String[] graph = new String[] {
-                "1 2 3",
-                "2 1 4 5",
-                "3 1 4",
-                "4 2 3 5 13",
-                "5 2 4 12 13",
-                "12 5 13",
-                "13 4 5 12",
-
-                "6 7 8",
-                "7 6 10 11",
-                "8 6 10",
-                "10 7 8 11",
-                "11 7 10",
-
-                "9" };
-
-        GiraphConfiguration conf = new GiraphConfiguration();
-        conf.setVertexClass(ConnectedComponentsVertex.class);
-        conf.setOutEdgesClass(ByteArrayEdges.class);
-        conf.setCombinerClass(MinimumIntCombiner.class);
-        conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
-        conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
-
-        // run internally
-        Iterable<String> results = InternalVertexRunner.run(conf, graph);
-
-        SetMultimap<Integer,Integer> components = parseResults(results);
-
-        Set<Integer> componentIDs = components.keySet();
-        assertEquals(3, componentIDs.size());
-        assertTrue(componentIDs.contains(1));
-        assertTrue(componentIDs.contains(6));
-        assertTrue(componentIDs.contains(9));
-
-        Set<Integer> componentOne = components.get(1);
-        assertEquals(7, componentOne.size());
-        assertTrue(componentOne.contains(1));
-        assertTrue(componentOne.contains(2));
-        assertTrue(componentOne.contains(3));
-        assertTrue(componentOne.contains(4));
-        assertTrue(componentOne.contains(5));
-        assertTrue(componentOne.contains(12));
-        assertTrue(componentOne.contains(13));
-
-        Set<Integer> componentTwo = components.get(6);
-        assertEquals(5, componentTwo.size());
-        assertTrue(componentTwo.contains(6));
-        assertTrue(componentTwo.contains(7));
-        assertTrue(componentTwo.contains(8));
-        assertTrue(componentTwo.contains(10));
-        assertTrue(componentTwo.contains(11));
-
-        Set<Integer> componentThree = components.get(9);
-        assertEquals(1, componentThree.size());
-        assertTrue(componentThree.contains(9));
-    }
-
-    private SetMultimap<Integer,Integer> parseResults(
-            Iterable<String> results) {
-        SetMultimap<Integer,Integer> components = HashMultimap.create();
-        for (String result : results) {
-            Iterable<String> parts = Splitter.on('\t').split(result);
-            int vertex = Integer.parseInt(Iterables.get(parts, 0));
-            int component = Integer.parseInt(Iterables.get(parts, 1));
-            components.put(component, vertex);
-        }
-        return components;
-    }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
deleted file mode 100644
index e4c029c..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.combiner.MinimumIntCombiner;
-import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.utils.InternalVertexRunner;
-import org.apache.giraph.utils.TestGraph;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.junit.Test;
-
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.SetMultimap;
-
-import java.util.AbstractMap.SimpleEntry;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- *  Tests for {@link ConnectedComponentsVertex}
- */
-public class ConnectedComponentsVertexTestInMemory {
-  public static Entry<IntWritable, NullWritable>[] makeEdges(int... args){
-    Entry<IntWritable, NullWritable> result[] =
-      new Entry[args.length];
-    for (int i=0; i<args.length; i++){
-      result[i] = new SimpleEntry<IntWritable, NullWritable>(
-          new IntWritable(args[i]), NullWritable.get());
-    }
-    return result;
-  }
-  /**
-   * A local integration test on toy data
-   */
-  @Test
-  public void testToyData() throws Exception {
-    GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(ConnectedComponentsVertex.class);
-    conf.setOutEdgesClass(ByteArrayEdges.class);
-    conf.setCombinerClass(MinimumIntCombiner.class);
-
-    TestGraph<IntWritable, IntWritable, NullWritable, IntWritable> graph =
-      new TestGraph<IntWritable, IntWritable, NullWritable, IntWritable> (conf);
-    // a small graph with three components
-    graph.addVertex(new IntWritable(1), new IntWritable(1), makeEdges(2, 3))
-      .addVertex(new IntWritable(2), new IntWritable(2), makeEdges(1, 4, 5))
-      .addVertex(new IntWritable(3), new IntWritable(3), makeEdges(1, 4))
-      .addVertex(new IntWritable(4), new IntWritable(4),
-          makeEdges(2, 3, 5, 13))
-      .addVertex(new IntWritable(5), new IntWritable(5),
-          makeEdges(2, 4, 12, 13))
-      .addVertex(new IntWritable(12), new IntWritable(12), makeEdges(5, 13))
-      .addVertex(new IntWritable(13), new IntWritable(13), makeEdges(4, 5, 12))
-      .addVertex(new IntWritable(6), new IntWritable(6), makeEdges(7, 8))
-      .addVertex(new IntWritable(7), new IntWritable(7), makeEdges(6, 10, 11))
-      .addVertex(new IntWritable(8), new IntWritable(8), makeEdges(6, 10))
-      .addVertex(new IntWritable(10), new IntWritable(10), makeEdges(7, 8, 11))
-      .addVertex(new IntWritable(11), new IntWritable(11), makeEdges(7, 10))
-      .addVertex(new IntWritable(9), new IntWritable(9));
-
-    // run internally
-    TestGraph<IntWritable, IntWritable, NullWritable, IntWritable> results =
-      InternalVertexRunner.run(conf, graph);
-
-    SetMultimap<Integer,Integer> components = parseResults(results);
-
-    Set<Integer> componentIDs = components.keySet();
-    assertEquals(3, componentIDs.size());
-    assertTrue(componentIDs.contains(1));
-    assertTrue(componentIDs.contains(6));
-    assertTrue(componentIDs.contains(9));
-
-    Set<Integer> componentOne = components.get(1);
-    assertEquals(7, componentOne.size());
-    assertTrue(componentOne.contains(1));
-    assertTrue(componentOne.contains(2));
-    assertTrue(componentOne.contains(3));
-    assertTrue(componentOne.contains(4));
-    assertTrue(componentOne.contains(5));
-    assertTrue(componentOne.contains(12));
-    assertTrue(componentOne.contains(13));
-
-    Set<Integer> componentTwo = components.get(6);
-    assertEquals(5, componentTwo.size());
-    assertTrue(componentTwo.contains(6));
-    assertTrue(componentTwo.contains(7));
-    assertTrue(componentTwo.contains(8));
-    assertTrue(componentTwo.contains(10));
-    assertTrue(componentTwo.contains(11));
-
-    Set<Integer> componentThree = components.get(9);
-    assertEquals(1, componentThree.size());
-    assertTrue(componentThree.contains(9));
-  }
-
-  private SetMultimap<Integer,Integer> parseResults(
-    TestGraph<IntWritable, IntWritable, NullWritable, IntWritable> results) {
-    SetMultimap<Integer,Integer> components = HashMultimap.create();
-    for (Vertex<IntWritable,
-                IntWritable,
-                NullWritable,
-                IntWritable> vertex: results) {
-      int component = vertex.getValue().get();
-      components.put(component, vertex.getId().get());
-    }
-    return components;
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankComputationTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankComputationTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankComputationTest.java
new file mode 100644
index 0000000..dcdef19
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankComputationTest.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.utils.InternalVertexRunner;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+
+/**
+ * Tests for {@link PageRankComputation}
+ */
+public class PageRankComputationTest {
+
+  /**
+   * A local integration test on toy data
+   */
+  @Test
+  public void testToyData() throws Exception {
+
+    // A small graph
+    String[] graph = new String[] {
+      "1 4 2 3",
+      "2 1",
+      "4 3 2",
+      "5 2 4"
+    };
+
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setInt(RandomWalkWithRestartComputation.MAX_SUPERSTEPS, 50);
+    conf.setFloat(
+        RandomWalkWithRestartComputation.TELEPORTATION_PROBABILITY, 0.15f);
+    conf.setComputationClass(PageRankComputation.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
+    conf.setVertexInputFormatClass(LongDoubleNullTextInputFormat.class);
+    conf.setVertexOutputFormatClass(
+        VertexWithDoubleValueNullEdgeTextOutputFormat.class);
+    conf.setWorkerContextClass(RandomWalkWorkerContext.class);
+    conf.setMasterComputeClass(RandomWalkVertexMasterCompute.class);
+    // Run internally
+    Iterable<String> results = InternalVertexRunner.run(conf, graph);
+
+    Map<Long, Double> steadyStateProbabilities =
+        RandomWalkTestUtils.parseSteadyStateProbabilities(results);
+
+    assertEquals(0.28159076008518047, steadyStateProbabilities.get(1l),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.2514648601529863, steadyStateProbabilities.get(2l),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.22262961972286327, steadyStateProbabilities.get(3l),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.17646783276703806, steadyStateProbabilities.get(4l),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.06784692727193153, steadyStateProbabilities.get(5l),
+        RandomWalkTestUtils.EPSILON);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
deleted file mode 100644
index 2a39291..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.utils.InternalVertexRunner;
-import org.junit.Test;
-
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-
-
-/**
- * Tests for {@link PageRankVertex}
- */
-public class PageRankVertexTest {
-
-  /**
-   * A local integration test on toy data
-   */
-  @Test
-  public void testToyData() throws Exception {
-
-    // A small graph
-    String[] graph = new String[] {
-      "1 4 2 3",
-      "2 1",
-      "4 3 2",
-      "5 2 4"
-    };
-
-    GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 50);
-    conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.15f);
-    conf.setVertexClass(PageRankVertex.class);
-    conf.setOutEdgesClass(ByteArrayEdges.class);
-    conf.setVertexInputFormatClass(LongDoubleNullTextInputFormat.class);
-    conf.setVertexOutputFormatClass(
-        VertexWithDoubleValueNullEdgeTextOutputFormat.class);
-    conf.setWorkerContextClass(RandomWalkWorkerContext.class);
-    conf.setMasterComputeClass(RandomWalkVertexMasterCompute.class);
-    // Run internally
-    Iterable<String> results = InternalVertexRunner.run(conf, graph);
-
-    Map<Long, Double> steadyStateProbabilities =
-        RandomWalkTestUtils.parseSteadyStateProbabilities(results);
-
-    assertEquals(0.28159076008518047, steadyStateProbabilities.get(1l),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.2514648601529863, steadyStateProbabilities.get(2l),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.22262961972286327, steadyStateProbabilities.get(3l),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.17646783276703806, steadyStateProbabilities.get(4l),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.06784692727193153, steadyStateProbabilities.get(5l),
-        RandomWalkTestUtils.EPSILON);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartComputationTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartComputationTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartComputationTest.java
new file mode 100644
index 0000000..ee27b71
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartComputationTest.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.utils.InternalVertexRunner;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link RandomWalkWithRestartComputation}
+ */
+public class RandomWalkWithRestartComputationTest {
+
+  /**
+   * A local integration test on toy data
+   */
+  @Test
+  public void testToyData() throws Exception {
+    // A small graph
+    String[] graph = new String[] { "12 34 56", "34 78", "56 34 78", "78 34" };
+
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setInt(RandomWalkWithRestartComputation.SOURCE_VERTEX, 12);
+    conf.setInt(RandomWalkWithRestartComputation.MAX_SUPERSTEPS, 30);
+    conf.setFloat(
+        RandomWalkWithRestartComputation.TELEPORTATION_PROBABILITY, 0.25f);
+    conf.setComputationClass(RandomWalkWithRestartComputation.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
+    conf.setVertexInputFormatClass(LongDoubleDoubleTextInputFormat.class);
+    conf.setVertexOutputFormatClass(
+        VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
+    conf.setWorkerContextClass(RandomWalkWorkerContext.class);
+    conf.setMasterComputeClass(RandomWalkVertexMasterCompute.class);
+    // Run internally
+    Iterable<String> results = InternalVertexRunner.run(conf, graph);
+
+    Map<Long, Double> steadyStateProbabilities =
+        RandomWalkTestUtils.parseSteadyStateProbabilities(results);
+    // values computed with external software
+    // 0.25, 0.354872, 0.09375, 0.301377
+    assertEquals(0.25, steadyStateProbabilities.get(12L), RandomWalkTestUtils.EPSILON);
+    assertEquals(0.354872, steadyStateProbabilities.get(34L),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.09375, steadyStateProbabilities.get(56L), RandomWalkTestUtils.EPSILON);
+    assertEquals(0.301377, steadyStateProbabilities.get(78L),
+        RandomWalkTestUtils.EPSILON);
+  }
+
+  /**
+   * A local integration test on toy data
+   */
+  @Test
+  public void testWeightedGraph() throws Exception {
+    // A small graph
+    String[] graph =
+        new String[] { "12 34:0.1 56:0.9", "34 78:0.9 56:0.1",
+          "56 12:0.1 34:0.8 78:0.1", "78 34:1.0" };
+
+    GiraphConfiguration conf = new GiraphConfiguration();
+    conf.setInt(RandomWalkWithRestartComputation.SOURCE_VERTEX, 12);
+    conf.setInt(RandomWalkWithRestartComputation.MAX_SUPERSTEPS, 30);
+    conf.setFloat(
+        RandomWalkWithRestartComputation.TELEPORTATION_PROBABILITY, 0.15f);
+    conf.setComputationClass(RandomWalkWithRestartComputation.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
+    conf.setVertexInputFormatClass(
+        NormalizingLongDoubleDoubleTextInputFormat.class);
+    conf.setVertexOutputFormatClass(
+        VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
+    conf.setWorkerContextClass(RandomWalkWorkerContext.class);
+    conf.setMasterComputeClass(RandomWalkVertexMasterCompute.class);
+    // Run internally
+    Iterable<String> results = InternalVertexRunner.run(conf, graph);
+
+    Map<Long, Double> steadyStateProbabilities =
+        RandomWalkTestUtils.parseSteadyStateProbabilities(results);
+    // values computed with external software
+    // 0.163365, 0.378932, 0.156886, 0.300816
+    assertEquals(0.163365, steadyStateProbabilities.get(12L),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.378932, steadyStateProbabilities.get(34L),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.156886, steadyStateProbabilities.get(56L),
+        RandomWalkTestUtils.EPSILON);
+    assertEquals(0.300816, steadyStateProbabilities.get(78L),
+        RandomWalkTestUtils.EPSILON);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
deleted file mode 100644
index a2a891e..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.utils.InternalVertexRunner;
-import org.junit.Test;
-
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests for {@link RandomWalkWithRestartVertex}
- */
-public class RandomWalkWithRestartVertexTest {
-
-  /**
-   * A local integration test on toy data
-   */
-  @Test
-  public void testToyData() throws Exception {
-    // A small graph
-    String[] graph = new String[] { "12 34 56", "34 78", "56 34 78", "78 34" };
-
-    GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setInt(RandomWalkWithRestartVertex.SOURCE_VERTEX, 12);
-    conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 30);
-    conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.25f);
-    conf.setVertexClass(RandomWalkWithRestartVertex.class);
-    conf.setOutEdgesClass(ByteArrayEdges.class);
-    conf.setVertexInputFormatClass(LongDoubleDoubleTextInputFormat.class);
-    conf.setVertexOutputFormatClass(
-        VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
-    conf.setWorkerContextClass(RandomWalkWorkerContext.class);
-    conf.setMasterComputeClass(RandomWalkVertexMasterCompute.class);
-    // Run internally
-    Iterable<String> results = InternalVertexRunner.run(conf, graph);
-
-    Map<Long, Double> steadyStateProbabilities =
-        RandomWalkTestUtils.parseSteadyStateProbabilities(results);
-    // values computed with external software
-    // 0.25, 0.354872, 0.09375, 0.301377
-    assertEquals(0.25, steadyStateProbabilities.get(12L), RandomWalkTestUtils.EPSILON);
-    assertEquals(0.354872, steadyStateProbabilities.get(34L),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.09375, steadyStateProbabilities.get(56L), RandomWalkTestUtils.EPSILON);
-    assertEquals(0.301377, steadyStateProbabilities.get(78L),
-        RandomWalkTestUtils.EPSILON);
-  }
-
-  /**
-   * A local integration test on toy data
-   */
-  @Test
-  public void testWeightedGraph() throws Exception {
-    // A small graph
-    String[] graph =
-        new String[] { "12 34:0.1 56:0.9", "34 78:0.9 56:0.1",
-          "56 12:0.1 34:0.8 78:0.1", "78 34:1.0" };
-
-    GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setInt(RandomWalkWithRestartVertex.SOURCE_VERTEX, 12);
-    conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 30);
-    conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.15f);
-    conf.setVertexClass(RandomWalkWithRestartVertex.class);
-    conf.setOutEdgesClass(ByteArrayEdges.class);
-    conf.setVertexInputFormatClass(
-        NormalizingLongDoubleDoubleTextInputFormat.class);
-    conf.setVertexOutputFormatClass(
-        VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
-    conf.setWorkerContextClass(RandomWalkWorkerContext.class);
-    conf.setMasterComputeClass(RandomWalkVertexMasterCompute.class);
-    // Run internally
-    Iterable<String> results = InternalVertexRunner.run(conf, graph);
-
-    Map<Long, Double> steadyStateProbabilities =
-        RandomWalkTestUtils.parseSteadyStateProbabilities(results);
-    // values computed with external software
-    // 0.163365, 0.378932, 0.156886, 0.300816
-    assertEquals(0.163365, steadyStateProbabilities.get(12L),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.378932, steadyStateProbabilities.get(34L),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.156886, steadyStateProbabilities.get(56L),
-        RandomWalkTestUtils.EPSILON);
-    assertEquals(0.300816, steadyStateProbabilities.get(78L),
-        RandomWalkTestUtils.EPSILON);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java
new file mode 100644
index 0000000..93a3248
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexInputFormat;
+import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexOutputFormat;
+import org.apache.giraph.utils.InternalVertexRunner;
+import org.apache.giraph.utils.MockUtils;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import java.util.Map;
+
+import static org.apache.giraph.examples.SimpleShortestPathsComputation.SOURCE_ID;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Contains a simple unit test for {@link SimpleShortestPathsComputation}
+ */
+public class SimpleShortestPathsComputationTest {
+
+  /**
+   * Test the behavior when a shorter path to a vertex has been found
+   */
+  @Test
+  public void testOnShorterPathFound() throws Exception {
+    Vertex<LongWritable, DoubleWritable, FloatWritable> vertex =
+        new Vertex<LongWritable, DoubleWritable, FloatWritable>();
+    SimpleShortestPathsComputation computation =
+        new SimpleShortestPathsComputation();
+    MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,
+        DoubleWritable> env = MockUtils.prepareVertexAndComputation(vertex,
+        new LongWritable(7L), new DoubleWritable(Double.MAX_VALUE), false,
+        computation, 1L);
+    Mockito.when(SOURCE_ID.get(env.getConfiguration())).thenReturn(2L);
+
+    vertex.addEdge(EdgeFactory.create(
+        new LongWritable(10L), new FloatWritable(2.5f)));
+    vertex.addEdge(EdgeFactory.create(
+        new LongWritable(20L), new FloatWritable(0.5f)));
+
+    computation.compute(vertex, Lists.newArrayList(new DoubleWritable(2),
+        new DoubleWritable(1.5)));
+
+    assertTrue(vertex.isHalted());
+    assertEquals(1.5d, vertex.getValue().get(), 0d);
+
+    env.verifyMessageSent(new LongWritable(10L), new DoubleWritable(4));
+    env.verifyMessageSent(new LongWritable(20L), new DoubleWritable(2));
+  }
+
+  /**
+   * Test the behavior when a new, but not shorter path to a vertex has been
+   * found.
+   */
+  @Test
+  public void testOnNoShorterPathFound() throws Exception {
+    Vertex<LongWritable, DoubleWritable, FloatWritable> vertex =
+        new Vertex<LongWritable, DoubleWritable, FloatWritable>();
+    SimpleShortestPathsComputation computation =
+        new SimpleShortestPathsComputation();
+    MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,
+        DoubleWritable> env = MockUtils.prepareVertexAndComputation(vertex,
+        new LongWritable(7L), new DoubleWritable(0.5), false, computation, 1L);
+    Mockito.when(SOURCE_ID.get(env.getConfiguration())).thenReturn(2L);
+
+    vertex.addEdge(EdgeFactory.create(new LongWritable(10L),
+        new FloatWritable(2.5f)));
+    vertex.addEdge(EdgeFactory.create(
+        new LongWritable(20L), new FloatWritable(0.5f)));
+
+    computation.compute(vertex, Lists.newArrayList(new DoubleWritable(2),
+        new DoubleWritable(1.5)));
+
+    assertTrue(vertex.isHalted());
+    assertEquals(0.5d, vertex.getValue().get(), 0d);
+
+    env.verifyNoMessageSent();
+  }
+
+  /**
+   * A local integration test on toy data
+   */
+  @Test
+  public void testToyData() throws Exception {
+
+    // a small four vertex graph
+    String[] graph = new String[] {
+        "[1,0,[[2,1],[3,3]]]",
+        "[2,0,[[3,1],[4,10]]]",
+        "[3,0,[[4,2]]]",
+        "[4,0,[]]"
+    };
+
+    GiraphConfiguration conf = new GiraphConfiguration();
+    // start from vertex 1
+    SOURCE_ID.set(conf, 1);
+    conf.setComputationClass(SimpleShortestPathsComputation.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
+    conf.setVertexInputFormatClass(
+        JsonLongDoubleFloatDoubleVertexInputFormat.class);
+    conf.setVertexOutputFormatClass(
+        JsonLongDoubleFloatDoubleVertexOutputFormat.class);
+
+    // run internally
+    Iterable<String> results = InternalVertexRunner.run(conf, graph);
+
+    Map<Long, Double> distances = parseDistances(results);
+
+    // verify results
+    assertNotNull(distances);
+    assertEquals(4, (int) distances.size());
+    assertEquals(0.0, (double) distances.get(1L), 0d);
+    assertEquals(1.0, (double) distances.get(2L), 0d);
+    assertEquals(2.0, (double) distances.get(3L), 0d);
+    assertEquals(4.0, (double) distances.get(4L), 0d);
+  }
+
+  private Map<Long, Double> parseDistances(Iterable<String> results) {
+    Map<Long, Double> distances =
+        Maps.newHashMapWithExpectedSize(Iterables.size(results));
+    for (String line : results) {
+      try {
+        JSONArray jsonVertex = new JSONArray(line);
+        distances.put(jsonVertex.getLong(0), jsonVertex.getDouble(1));
+      } catch (JSONException e) {
+        throw new IllegalArgumentException(
+            "Couldn't get vertex from line " + line, e);
+      }
+    }
+    return distances;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
deleted file mode 100644
index 1006577..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.conf.GiraphConfiguration;
-import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexInputFormat;
-import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexOutputFormat;
-import org.apache.giraph.utils.InternalVertexRunner;
-import org.apache.giraph.utils.MockUtils;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-import java.util.Map;
-
-import static org.apache.giraph.examples.SimpleShortestPathsVertex.SOURCE_ID;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Contains a simple unit test for {@link SimpleShortestPathsVertex}
- */
-public class SimpleShortestPathsVertexTest {
-
-  /**
-   * Test the behavior when a shorter path to a vertex has been found
-   */
-  @Test
-  public void testOnShorterPathFound() throws Exception {
-    SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex();
-
-    MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,
-    DoubleWritable> env = MockUtils.prepareVertex(vertex, 1L,
-        new LongWritable(7L), new DoubleWritable(Double.MAX_VALUE), false);
-
-    Mockito.when(SOURCE_ID.get(env.getConfiguration())).thenReturn(2L);
-
-    vertex.addEdge(EdgeFactory.create(
-        new LongWritable(10L), new FloatWritable(2.5f)));
-    vertex.addEdge(EdgeFactory.create(
-        new LongWritable(20L), new FloatWritable(0.5f)));
-
-    vertex.compute(Lists.newArrayList(new DoubleWritable(2),
-        new DoubleWritable(1.5)));
-
-    assertTrue(vertex.isHalted());
-    assertEquals(1.5d, vertex.getValue().get(), 0d);
-
-    env.verifyMessageSent(new LongWritable(10L), new DoubleWritable(4));
-    env.verifyMessageSent(new LongWritable(20L), new DoubleWritable(2));
-  }
-
-  /**
-   * Test the behavior when a new, but not shorter path to a vertex has been
-   * found.
-   */
-  @Test
-  public void testOnNoShorterPathFound() throws Exception {
-
-    SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex();
-
-    MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,
-    DoubleWritable> env = MockUtils.prepareVertex(vertex, 1L,
-        new LongWritable(7L), new DoubleWritable(0.5), false);
-
-    Mockito.when(SOURCE_ID.get(env.getConfiguration())).thenReturn(2L);
-
-    vertex.addEdge(EdgeFactory.create(new LongWritable(10L),
-        new FloatWritable(2.5f)));
-    vertex.addEdge(EdgeFactory.create(
-        new LongWritable(20L), new FloatWritable(0.5f)));
-
-    vertex.compute(Lists.newArrayList(new DoubleWritable(2),
-        new DoubleWritable(1.5)));
-
-    assertTrue(vertex.isHalted());
-    assertEquals(0.5d, vertex.getValue().get(), 0d);
-
-    env.verifyNoMessageSent();
-  }
-
-  /**
-   * A local integration test on toy data
-   */
-  @Test
-  public void testToyData() throws Exception {
-
-    // a small four vertex graph
-    String[] graph = new String[] {
-        "[1,0,[[2,1],[3,3]]]",
-        "[2,0,[[3,1],[4,10]]]",
-        "[3,0,[[4,2]]]",
-        "[4,0,[]]"
-    };
-
-    GiraphConfiguration conf = new GiraphConfiguration();
-    // start from vertex 1
-    SOURCE_ID.set(conf, 1);
-    conf.setVertexClass(SimpleShortestPathsVertex.class);
-    conf.setOutEdgesClass(ByteArrayEdges.class);
-    conf.setVertexInputFormatClass(
-        JsonLongDoubleFloatDoubleVertexInputFormat.class);
-    conf.setVertexOutputFormatClass(
-        JsonLongDoubleFloatDoubleVertexOutputFormat.class);
-
-    // run internally
-    Iterable<String> results = InternalVertexRunner.run(conf, graph);
-
-    Map<Long, Double> distances = parseDistances(results);
-
-    // verify results
-    assertNotNull(distances);
-    assertEquals(4, (int) distances.size());
-    assertEquals(0.0, (double) distances.get(1L), 0d);
-    assertEquals(1.0, (double) distances.get(2L), 0d);
-    assertEquals(2.0, (double) distances.get(3L), 0d);
-    assertEquals(4.0, (double) distances.get(4L), 0d);
-  }
-
-  private Map<Long, Double> parseDistances(Iterable<String> results) {
-    Map<Long, Double> distances =
-        Maps.newHashMapWithExpectedSize(Iterables.size(results));
-    for (String line : results) {
-      try {
-        JSONArray jsonVertex = new JSONArray(line);
-        distances.put(jsonVertex.getLong(0), jsonVertex.getDouble(1));
-      } catch (JSONException e) {
-        throw new IllegalArgumentException(
-            "Couldn't get vertex from line " + line, e);
-      }
-    }
-    return distances;
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java
new file mode 100644
index 0000000..73516a4
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import com.google.common.collect.Lists;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.MockUtils;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.junit.Test;
+
+import static org.apache.giraph.examples.SimpleTriangleClosingComputation.IntArrayListWritable;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Contains a simple unit test for {@link SimpleTriangleClosingComputation}
+ */
+public class SimpleTriangleClosingComputationTest {
+
+  /**
+   * Test the behavior of the triangle closing algorithm:
+   * does it send all its out edge values to all neighbors?
+   */
+  @Test
+  public void testSuperstepZero() throws Exception {
+    // this guy should end up with an array value of 4
+    Vertex<IntWritable, IntArrayListWritable, NullWritable> vertex =
+        new Vertex<IntWritable, IntArrayListWritable, NullWritable>();
+
+    IntArrayListWritable alw = new IntArrayListWritable();
+
+    SimpleTriangleClosingComputation computation =
+        new SimpleTriangleClosingComputation();
+    MockUtils.MockedEnvironment env = MockUtils.prepareVertexAndComputation(
+        vertex, new IntWritable(1), alw, false, computation, 0L);
+
+    vertex.addEdge(EdgeFactory.create(new IntWritable(5)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(7)));
+
+    computation.compute(vertex, Lists.<IntWritable>newArrayList(
+      new IntWritable(83), new IntWritable(42)));
+
+    env.verifyMessageSent(new IntWritable(5), new IntWritable(5));
+    env.verifyMessageSent(new IntWritable(5), new IntWritable(7));
+    env.verifyMessageSent(new IntWritable(7), new IntWritable(5));
+    env.verifyMessageSent(new IntWritable(7), new IntWritable(7));
+  }
+
+  /** Test behavior of compute() with incoming messages (superstep 1) */
+  @Test
+  public void testSuperstepOne() throws Exception {
+    // see if the vertex interprets its incoming
+    // messages properly to verify the algorithm
+    Vertex<IntWritable, IntArrayListWritable, NullWritable> vertex =
+        new Vertex<IntWritable, IntArrayListWritable, NullWritable>();
+    SimpleTriangleClosingComputation computation =
+        new SimpleTriangleClosingComputation();
+    MockUtils.MockedEnvironment env = MockUtils.prepareVertexAndComputation(
+        vertex, new IntWritable(1), null, false, computation, 1L);
+
+      // superstep 1: can the vertex process these correctly?
+      computation.compute(vertex, Lists.<IntWritable>newArrayList(
+        new IntWritable(7),
+        new IntWritable(3),
+        new IntWritable(4),
+        new IntWritable(7),
+        new IntWritable(4),
+        new IntWritable(2),
+        new IntWritable(4)));
+      final String pairCheck = "[4, 7]";
+      assertEquals(pairCheck, vertex.getValue().toString());
+  }
+ }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
deleted file mode 100644
index bfe8002..0000000
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import com.google.common.collect.Lists;
-import org.apache.giraph.utils.MockUtils;
-import org.apache.giraph.edge.EdgeFactory;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Contains a simple unit test for {@link SimpleTriangleClosingVertex}
- */
-public class SimpleTriangleClosingVertexTest {
-
-  /**
-   * Test the behavior of the triangle closing algorithm:
-   * does it send all its out edge values to all neighbors?
-   */
-  @Test
-  public void testSuperstepZero() throws Exception {
-    // this guy should end up with an array value of 4
-    SimpleTriangleClosingVertex vertex =
-        new SimpleTriangleClosingVertex();
-
-    SimpleTriangleClosingVertex.IntArrayListWritable alw =
-      new SimpleTriangleClosingVertex.IntArrayListWritable();
-
-    MockUtils.MockedEnvironment<IntWritable,
-      SimpleTriangleClosingVertex.IntArrayListWritable,
-    NullWritable, IntWritable> env =
-      MockUtils.prepareVertex(vertex, 0L,
-        new IntWritable(1), alw, false);
-
-    vertex.addEdge(EdgeFactory.create(new IntWritable(5)));
-    vertex.addEdge(EdgeFactory.create(new IntWritable(7)));
-
-    vertex.compute(Lists.<IntWritable>newArrayList(
-      new IntWritable(83), new IntWritable(42)));
-
-    env.verifyMessageSent(new IntWritable(5), new IntWritable(5));
-    env.verifyMessageSent(new IntWritable(5), new IntWritable(7));
-    env.verifyMessageSent(new IntWritable(7), new IntWritable(5));
-    env.verifyMessageSent(new IntWritable(7), new IntWritable(7));
-  }
-
-  /** Test behavior of compute() with incoming messages (superstep 1) */
-  @Test
-  public void testSuperstepOne() throws Exception {
-    // see if the vertex interprets its incoming
-    // messages properly to verify the algorithm
-    SimpleTriangleClosingVertex vertex =
-      new SimpleTriangleClosingVertex();
-
-    MockUtils.MockedEnvironment<IntWritable,
-      SimpleTriangleClosingVertex.IntArrayListWritable,
-      NullWritable, IntWritable>
-      env = MockUtils.<IntWritable,
-      SimpleTriangleClosingVertex.IntArrayListWritable,
-      NullWritable, IntWritable> prepareVertex(
-        vertex, 1L, new IntWritable(1), null, false);
-      // superstep 1: can the vertex process these correctly?
-      vertex.compute(Lists.<IntWritable>newArrayList(
-        new IntWritable(7),
-        new IntWritable(3),
-        new IntWritable(4),
-        new IntWritable(7),
-        new IntWritable(4),
-        new IntWritable(2),
-        new IntWritable(4)));
-      final String pairCheck = "[4, 7]";
-      assertEquals(pairCheck, vertex.getValue().toString());
-  }
- }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/TestPageRank.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/TestPageRank.java b/giraph-examples/src/test/java/org/apache/giraph/examples/TestPageRank.java
index 922e736..f8c7c48 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/TestPageRank.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/TestPageRank.java
@@ -64,13 +64,13 @@ public class TestPageRank extends BspCase {
   private void testPageRank(int numComputeThreads)
       throws IOException, InterruptedException, ClassNotFoundException {
     GiraphConfiguration conf = new GiraphConfiguration();
-    conf.setVertexClass(SimplePageRankVertex.class);
+    conf.setComputationClass(SimplePageRankComputation.class);
     conf.setVertexInputFormatClass(
-        SimplePageRankVertex.SimplePageRankVertexInputFormat.class);
+        SimplePageRankComputation.SimplePageRankVertexInputFormat.class);
     conf.setWorkerContextClass(
-        SimplePageRankVertex.SimplePageRankVertexWorkerContext.class);
+        SimplePageRankComputation.SimplePageRankWorkerContext.class);
     conf.setMasterComputeClass(
-        SimplePageRankVertex.SimplePageRankVertexMasterCompute.class);
+        SimplePageRankComputation.SimplePageRankMasterCompute.class);
     conf.setNumComputeThreads(numComputeThreads);
     // Set enough partitions to generate randomness on the compute side
     if (numComputeThreads != 1) {
@@ -80,11 +80,11 @@ public class TestPageRank extends BspCase {
     assertTrue(job.run(true));
     if (!runningInDistributedMode()) {
       double maxPageRank =
-          SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMax();
+          SimplePageRankComputation.SimplePageRankWorkerContext.getFinalMax();
       double minPageRank =
-          SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalMin();
+          SimplePageRankComputation.SimplePageRankWorkerContext.getFinalMin();
       long numVertices =
-          SimplePageRankVertex.SimplePageRankVertexWorkerContext.getFinalSum();
+          SimplePageRankComputation.SimplePageRankWorkerContext.getFinalSum();
       System.out.println(getCallingMethodName() + ": maxPageRank=" +
           maxPageRank + " minPageRank=" +
           minPageRank + " numVertices=" + numVertices + ", " +

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
index 23d055e..1323ff6 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/TryMultiIpcBindingPortsTest.java
@@ -70,7 +70,7 @@ public class TryMultiIpcBindingPortsTest {
         // fail the first port binding attempt
         GiraphConfiguration conf = new GiraphConfiguration();
         GiraphConstants.FAIL_FIRST_IPC_PORT_BIND_ATTEMPT.set(conf, true);
-        conf.setVertexClass(ConnectedComponentsVertex.class);
+        conf.setComputationClass(ConnectedComponentsComputation.class);
         conf.setOutEdgesClass(ByteArrayEdges.class);
         conf.setCombinerClass(MinimumIntCombiner.class);
         conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/test/java/org/apache/giraph/vertex/TestComputationTypes.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/vertex/TestComputationTypes.java b/giraph-examples/src/test/java/org/apache/giraph/vertex/TestComputationTypes.java
new file mode 100644
index 0000000..3c63ee0
--- /dev/null
+++ b/giraph-examples/src/test/java/org/apache/giraph/vertex/TestComputationTypes.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.vertex;
+
+import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.examples.SimpleSuperstepComputation.SimpleSuperstepVertexInputFormat;
+import org.apache.giraph.graph.VertexValueFactory;
+import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
+import org.apache.giraph.io.formats.JsonBase64VertexInputFormat;
+import org.apache.giraph.io.formats.JsonBase64VertexOutputFormat;
+import org.apache.giraph.job.GiraphConfigurationValidator;
+import org.apache.giraph.utils.NoOpComputation;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Test;
+
+import static org.apache.giraph.conf.GiraphConstants.VERTEX_VALUE_FACTORY_CLASS;
+
+
+public class TestComputationTypes {
+
+    /**
+     * Matches the {@link GeneratedVertexInputFormat}
+     */
+    private static class GeneratedComputationMatch extends NoOpComputation<
+        LongWritable, IntWritable, FloatWritable, FloatWritable> { }
+
+    /**
+     * Matches the {@link GeneratedVertexInputFormat}
+     */
+    private static class DerivedComputationMatch extends
+        GeneratedComputationMatch {
+    }
+
+    /**
+     * Mismatches the {@link GeneratedVertexInputFormat}
+     */
+    private static class GeneratedComputationMismatch extends NoOpComputation<
+        LongWritable, FloatWritable, FloatWritable, FloatWritable> { }
+
+    /**
+     * Matches the {@link GeneratedComputationMatch}
+     */
+    private static class GeneratedVertexMatchCombiner extends
+        Combiner<LongWritable, FloatWritable> {
+      @Override
+      public void combine(LongWritable vertexIndex,
+          FloatWritable originalMessage,
+          FloatWritable messageToCombine) {
+      }
+
+      @Override
+      public FloatWritable createInitialMessage() {
+        return null;
+      }
+    }
+
+    /**
+     * Mismatches the {@link GeneratedComputationMatch}
+     */
+    private static class GeneratedVertexMismatchCombiner extends
+        Combiner<LongWritable, DoubleWritable> {
+      @Override
+      public void combine(LongWritable vertexIndex,
+          DoubleWritable originalMessage,
+          DoubleWritable messageToCombine) {
+      }
+
+      @Override
+      public DoubleWritable createInitialMessage() {
+        return null;
+      }
+    }
+
+    /**
+     * Mismatches the {@link GeneratedComputationMatch}
+     */
+    private static class GeneratedVertexMismatchValueFactory implements
+        VertexValueFactory<DoubleWritable> {
+
+      @Override
+      public void initialize(
+          ImmutableClassesGiraphConfiguration<?, DoubleWritable, ?>
+              configuration) {}
+
+      @Override
+      public DoubleWritable createVertexValue() {
+        return new DoubleWritable();
+      }
+    }
+
+    /**
+     * Just populate a conf with testing defaults that won't
+     * upset the GiraphConfigurationValidator.
+     * */
+    private Configuration getDefaultTestConf() {
+      Configuration conf = new Configuration();
+      conf.setInt(GiraphConstants.MAX_WORKERS, 1);
+      conf.setInt(GiraphConstants.MIN_WORKERS, 1);
+      conf.set(GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.getKey(),
+        "org.apache.giraph.io.formats.DUMMY_TEST_VALUE");
+      return conf;
+    }
+
+    @Test
+    public void testMatchingType() throws SecurityException,
+            NoSuchMethodException, NoSuchFieldException {
+        Configuration conf = getDefaultTestConf();
+        GiraphConstants.COMPUTATION_CLASS.set(conf,
+            GeneratedComputationMatch.class);
+        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
+        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
+            SimpleSuperstepVertexInputFormat.class);
+        GiraphConstants.VERTEX_COMBINER_CLASS.set(conf,
+            GeneratedVertexMatchCombiner.class);
+      @SuppressWarnings("rawtypes")
+      GiraphConfigurationValidator<?, ?, ?, ?, ?> validator =
+        new GiraphConfigurationValidator(conf);
+
+      ImmutableClassesGiraphConfiguration gc = new
+          ImmutableClassesGiraphConfiguration(conf);
+
+
+      validator.validateConfiguration();
+    }
+
+    @Test
+    public void testDerivedMatchingType() throws SecurityException,
+            NoSuchMethodException, NoSuchFieldException {
+        Configuration conf = getDefaultTestConf() ;
+        GiraphConstants.COMPUTATION_CLASS.set(conf,
+            DerivedComputationMatch.class);
+        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
+        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
+            SimpleSuperstepVertexInputFormat.class);
+        @SuppressWarnings("rawtypes")
+        GiraphConfigurationValidator<?, ?, ?, ?, ?> validator =
+          new GiraphConfigurationValidator(conf);
+        validator.validateConfiguration();
+    }
+
+    @Test
+    public void testDerivedInputFormatType() throws SecurityException,
+            NoSuchMethodException, NoSuchFieldException {
+        Configuration conf = getDefaultTestConf() ;
+        GiraphConstants.COMPUTATION_CLASS.set(conf,
+            DerivedComputationMatch.class);
+        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
+        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
+            SimpleSuperstepVertexInputFormat.class);
+      @SuppressWarnings("rawtypes")
+      GiraphConfigurationValidator<?, ?, ?, ?, ?> validator =
+        new GiraphConfigurationValidator(conf);
+      validator.validateConfiguration();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testMismatchingVertex() throws SecurityException,
+      NoSuchMethodException, NoSuchFieldException {
+      Configuration conf = getDefaultTestConf() ;
+      GiraphConstants.COMPUTATION_CLASS.set(conf,
+          GeneratedComputationMismatch.class);
+      GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
+      GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
+        SimpleSuperstepVertexInputFormat.class);
+      @SuppressWarnings("rawtypes")
+      GiraphConfigurationValidator<?, ?, ?, ?, ?> validator =
+        new GiraphConfigurationValidator(conf);
+      validator.validateConfiguration();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testMismatchingCombiner() throws SecurityException,
+      NoSuchMethodException, NoSuchFieldException {
+      Configuration conf = getDefaultTestConf() ;
+      GiraphConstants.COMPUTATION_CLASS.set(conf,
+          GeneratedComputationMatch.class);
+      GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
+      GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
+        SimpleSuperstepVertexInputFormat.class);
+      GiraphConstants.VERTEX_COMBINER_CLASS.set(conf,
+        GeneratedVertexMismatchCombiner.class);
+      @SuppressWarnings("rawtypes")
+      GiraphConfigurationValidator<?, ?, ?, ?, ?> validator =
+        new GiraphConfigurationValidator(conf);
+      validator.validateConfiguration();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testMismatchingVertexValueFactory() throws SecurityException,
+        NoSuchMethodException, NoSuchFieldException {
+      Configuration conf = getDefaultTestConf() ;
+      GiraphConstants.COMPUTATION_CLASS.set(conf,
+          GeneratedComputationMatch.class);
+      GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
+      GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
+          SimpleSuperstepVertexInputFormat.class);
+      VERTEX_VALUE_FACTORY_CLASS.set(conf,
+          GeneratedVertexMismatchValueFactory.class);
+      @SuppressWarnings("rawtypes")
+      GiraphConfigurationValidator<?, ?, ?, ?, ?> validator =
+          new GiraphConfigurationValidator(conf);
+      validator.validateConfiguration();
+    }
+
+    @Test
+    public void testJsonBase64FormatType() throws SecurityException,
+            NoSuchMethodException, NoSuchFieldException {
+        Configuration conf = getDefaultTestConf() ;
+        GiraphConstants.COMPUTATION_CLASS.set(conf,
+            GeneratedComputationMatch.class);
+        GiraphConstants.VERTEX_EDGES_CLASS.set(conf, ByteArrayEdges.class);
+        GiraphConstants.VERTEX_INPUT_FORMAT_CLASS.set(conf,
+            JsonBase64VertexInputFormat.class);
+        GiraphConstants.VERTEX_OUTPUT_FORMAT_CLASS.set(conf,
+            JsonBase64VertexOutputFormat.class);
+        @SuppressWarnings("rawtypes")
+        GiraphConfigurationValidator<?, ?, ?, ?, ?> validator =
+          new GiraphConfigurationValidator(conf);
+        validator.validateConfiguration();
+    }
+}


[12/12] git commit: updated refs/heads/trunk to 8811165

Posted by ma...@apache.org.
GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable  (majakabiljo)


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/8811165e
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/8811165e
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/8811165e

Branch: refs/heads/trunk
Commit: 8811165e85ae2db442e34f1021db29f4dfcc8430
Parents: 9f7a347
Author: Maja Kabiljo <ma...@maja-mbp.thefacebook.com>
Authored: Mon May 20 10:24:19 2013 -0700
Committer: Maja Kabiljo <ma...@maja-mbp.thefacebook.com>
Committed: Mon May 20 10:26:04 2013 -0700

----------------------------------------------------------------------
 CHANGELOG                                          |    3 +
 .../io/accumulo/AccumuloVertexInputFormat.java     |    6 +-
 .../io/accumulo/TestAccumuloVertexFormat.java      |   18 +-
 .../edgemarker/AccumuloEdgeInputFormat.java        |    4 +-
 .../edgemarker/AccumuloEdgeOutputFormat.java       |    2 +-
 .../giraph/benchmark/AggregatorsBenchmark.java     |   14 +-
 .../apache/giraph/benchmark/PageRankBenchmark.java |    6 +-
 .../giraph/benchmark/PageRankComputation.java      |   58 +++
 .../apache/giraph/benchmark/PageRankVertex.java    |   55 ---
 .../giraph/benchmark/RandomMessageBenchmark.java   |   23 +-
 .../giraph/benchmark/ShortestPathsBenchmark.java   |    4 +-
 .../giraph/benchmark/ShortestPathsComputation.java |   76 ++++
 .../giraph/benchmark/ShortestPathsVertex.java      |   65 ----
 .../benchmark/WeightedPageRankBenchmark.java       |    6 +-
 .../benchmark/WeightedPageRankComputation.java     |   72 ++++
 .../giraph/benchmark/WeightedPageRankVertex.java   |   66 ----
 .../java/org/apache/giraph/bsp/BspService.java     |   22 +-
 .../org/apache/giraph/bsp/CentralizedService.java  |    3 +-
 .../giraph/bsp/CentralizedServiceMaster.java       |    5 +-
 .../giraph/bsp/CentralizedServiceWorker.java       |   20 +-
 .../java/org/apache/giraph/comm/SendCache.java     |    6 +-
 .../java/org/apache/giraph/comm/SendEdgeCache.java |    2 +-
 .../org/apache/giraph/comm/SendMessageCache.java   |    5 +-
 .../org/apache/giraph/comm/SendMutationsCache.java |   37 +-
 .../org/apache/giraph/comm/SendPartitionCache.java |   22 +-
 .../java/org/apache/giraph/comm/ServerData.java    |   63 ++--
 .../java/org/apache/giraph/comm/WorkerClient.java  |    3 +-
 .../giraph/comm/WorkerClientRequestProcessor.java  |   11 +-
 .../java/org/apache/giraph/comm/WorkerServer.java  |   10 +-
 .../messages/ByteArrayMessagesPerVertexStore.java  |   29 +-
 .../comm/messages/DiskBackedMessageStore.java      |   25 +-
 .../DiskBackedMessageStoreByPartition.java         |   30 +-
 .../comm/messages/InMemoryMessageStoreFactory.java |   76 ++++
 .../giraph/comm/messages/MessageStoreFactory.java  |    8 +-
 .../comm/messages/OneMessagePerVertexStore.java    |   32 +-
 .../comm/messages/SequentialFileMessageStore.java  |   24 +-
 .../giraph/comm/messages/SimpleMessageStore.java   |   13 +-
 .../giraph/comm/netty/NettyMasterClient.java       |    4 +-
 .../giraph/comm/netty/NettyMasterServer.java       |    2 +-
 .../NettyWorkerAggregatorRequestProcessor.java     |    8 +-
 .../giraph/comm/netty/NettyWorkerClient.java       |   15 +-
 .../netty/NettyWorkerClientRequestProcessor.java   |   84 ++--
 .../giraph/comm/netty/NettyWorkerServer.java       |   78 ++---
 .../netty/handler/WorkerRequestServerHandler.java  |   18 +-
 .../SendPartitionCurrentMessagesRequest.java       |   11 +-
 .../requests/SendPartitionMutationsRequest.java    |   23 +-
 .../giraph/comm/requests/SendVertexRequest.java    |   11 +-
 .../comm/requests/SendWorkerMessagesRequest.java   |    3 +-
 .../apache/giraph/comm/requests/WorkerRequest.java |    5 +-
 .../giraph/comm/requests/WritableRequest.java      |   13 +-
 .../java/org/apache/giraph/conf/AllOptions.java    |    4 +-
 .../DefaultImmutableClassesGiraphConfigurable.java |   11 +-
 .../java/org/apache/giraph/conf/GiraphClasses.java |  176 +++++-----
 .../apache/giraph/conf/GiraphConfiguration.java    |   34 +--
 .../org/apache/giraph/conf/GiraphConstants.java    |   26 +-
 .../conf/ImmutableClassesGiraphConfigurable.java   |    9 +-
 .../conf/ImmutableClassesGiraphConfiguration.java  |  178 +++++-----
 .../apache/giraph/edge/ConfigurableOutEdges.java   |    4 +-
 .../java/org/apache/giraph/edge/EdgeStore.java     |   15 +-
 .../apache/giraph/edge/MutableEdgesIterable.java   |    4 +-
 .../apache/giraph/edge/MutableEdgesWrapper.java    |    2 +-
 .../org/apache/giraph/graph/BasicComputation.java  |   35 ++
 .../java/org/apache/giraph/graph/Computation.java  |  260 +++++++++++++
 .../org/apache/giraph/graph/ComputeCallable.java   |   80 ++---
 .../apache/giraph/graph/DefaultVertexResolver.java |   69 ++---
 .../giraph/graph/DefaultVertexValueFactory.java    |    2 +-
 .../giraph/graph/GiraphTransferRegulator.java      |    5 +-
 .../java/org/apache/giraph/graph/GraphMapper.java  |    7 +-
 .../java/org/apache/giraph/graph/GraphState.java   |   64 +---
 .../org/apache/giraph/graph/GraphStateAware.java   |   47 ---
 .../org/apache/giraph/graph/GraphTaskManager.java  |  110 +++---
 .../main/java/org/apache/giraph/graph/Vertex.java  |  193 +----------
 .../org/apache/giraph/graph/VertexChanges.java     |    5 +-
 .../org/apache/giraph/graph/VertexMutations.java   |   22 +-
 .../org/apache/giraph/graph/VertexResolver.java    |   17 +-
 .../apache/giraph/graph/VertexValueFactory.java    |    2 +-
 .../SuperstepHashPartitionerFactory.java           |   14 +-
 .../main/java/org/apache/giraph/io/EdgeReader.java |    2 +-
 .../org/apache/giraph/io/GiraphInputFormat.java    |    2 +-
 .../org/apache/giraph/io/SimpleVertexWriter.java   |    2 +-
 .../org/apache/giraph/io/VertexOutputFormat.java   |    2 +-
 .../java/org/apache/giraph/io/VertexReader.java    |    4 +-
 .../org/apache/giraph/io/VertexValueReader.java    |    4 +-
 .../java/org/apache/giraph/io/VertexWriter.java    |    2 +-
 .../giraph/io/filters/DefaultEdgeInputFilter.java  |    2 +-
 .../io/filters/DefaultVertexInputFilter.java       |    6 +-
 .../giraph/io/filters/VertexInputFilter.java       |    5 +-
 .../AdjacencyListTextVertexOutputFormat.java       |    2 +-
 .../io/formats/IdWithValueTextOutputFormat.java    |    2 +-
 .../formats/IntIntTextVertexValueInputFormat.java  |    6 +-
 .../io/formats/JsonBase64VertexOutputFormat.java   |    2 +-
 ...JsonLongDoubleFloatDoubleVertexInputFormat.java |    5 +-
 ...sonLongDoubleFloatDoubleVertexOutputFormat.java |    3 +-
 .../PseudoRandomIntNullVertexInputFormat.java      |    4 +-
 .../io/formats/PseudoRandomVertexInputFormat.java  |    4 +-
 .../io/formats/SequenceFileVertexInputFormat.java  |    6 +-
 .../io/formats/SequenceFileVertexOutputFormat.java |    2 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |    5 +-
 .../giraph/io/formats/TextVertexInputFormat.java   |   14 +-
 .../giraph/io/formats/TextVertexOutputFormat.java  |    2 +-
 .../io/formats/TextVertexValueInputFormat.java     |    3 +-
 .../formats/multi/EdgeInputFormatDescription.java  |   12 +-
 .../io/formats/multi/MultiEdgeInputFormat.java     |    2 +-
 .../io/formats/multi/MultiVertexInputFormat.java   |    2 +-
 .../multi/VertexInputFormatDescription.java        |   12 +-
 .../giraph/io/internal/WrappedEdgeReader.java      |    4 +-
 .../io/internal/WrappedVertexOutputFormat.java     |    4 +-
 .../giraph/io/internal/WrappedVertexReader.java    |    6 +-
 .../giraph/io/iterables/EdgeReaderWrapper.java     |    2 +-
 .../giraph/io/iterables/VertexReaderWrapper.java   |   12 +-
 .../MultiThreadedSuperstepOutput.java              |    8 +-
 .../io/superstep_output/NoOpSuperstepOutput.java   |    2 +-
 .../SynchronizedSuperstepOutput.java               |    7 +-
 .../giraph/job/GiraphConfigurationValidator.java   |   52 ++--
 .../org/apache/giraph/master/BspServiceMaster.java |   48 ++-
 .../giraph/master/MasterAggregatorHandler.java     |    4 +-
 .../org/apache/giraph/master/MasterCompute.java    |   98 +++---
 .../org/apache/giraph/master/MasterThread.java     |    7 +-
 .../org/apache/giraph/master/SuperstepClasses.java |  160 ++++++++
 .../apache/giraph/partition/BasicPartition.java    |   20 +-
 .../giraph/partition/ByteArrayPartition.java       |   29 +-
 .../giraph/partition/DefaultPartitionContext.java  |   34 --
 .../giraph/partition/DiskBackedPartitionStore.java |   78 ++--
 .../giraph/partition/GraphPartitionerFactory.java  |    7 +-
 .../giraph/partition/HashMasterPartitioner.java    |    5 +-
 .../giraph/partition/HashPartitionerFactory.java   |   13 +-
 .../partition/HashRangePartitionerFactory.java     |   15 +-
 .../partition/HashRangeWorkerPartitioner.java      |    5 +-
 .../giraph/partition/HashWorkerPartitioner.java    |    9 +-
 .../giraph/partition/MasterGraphPartitioner.java   |    3 +-
 .../org/apache/giraph/partition/Partition.java     |   24 +-
 .../apache/giraph/partition/PartitionContext.java  |   45 ---
 .../apache/giraph/partition/PartitionStore.java    |   13 +-
 .../giraph/partition/RangeMasterPartitioner.java   |    5 +-
 .../giraph/partition/RangePartitionerFactory.java  |    5 +-
 .../apache/giraph/partition/RangeSplitHint.java    |    2 +-
 .../giraph/partition/RangeWorkerPartitioner.java   |    5 +-
 .../giraph/partition/ReusesObjectsPartition.java   |    4 +-
 .../SimpleIntRangePartitionerFactory.java          |   13 +-
 .../SimpleLongRangePartitionerFactory.java         |   13 +-
 .../apache/giraph/partition/SimplePartition.java   |   31 +-
 .../giraph/partition/SimplePartitionStore.java     |   21 +-
 .../partition/SimpleRangeMasterPartitioner.java    |    5 +-
 .../partition/SimpleRangeWorkerPartitioner.java    |   10 +-
 .../giraph/partition/WorkerGraphPartitioner.java   |    7 +-
 .../apache/giraph/utils/ByteArrayVertexIdData.java |    4 +-
 .../giraph/utils/ByteArrayVertexIdEdges.java       |    4 +-
 .../giraph/utils/ByteArrayVertexIdMessages.java    |   26 +-
 .../apache/giraph/utils/ConfigurationUtils.java    |   17 +-
 .../giraph/utils/InMemoryVertexInputFormat.java    |    6 +-
 .../apache/giraph/utils/InternalVertexRunner.java  |   14 +-
 .../org/apache/giraph/utils/ReflectionUtils.java   |   20 +
 .../java/org/apache/giraph/utils/TestGraph.java    |   32 +-
 .../org/apache/giraph/utils/VertexIdIterator.java  |    2 +-
 .../org/apache/giraph/utils/WritableUtils.java     |  109 ++++--
 .../org/apache/giraph/worker/BspServiceWorker.java |  125 +++----
 .../giraph/worker/EdgeInputSplitsCallable.java     |   25 +-
 .../worker/EdgeInputSplitsCallableFactory.java     |   22 +-
 .../apache/giraph/worker/InputSplitsCallable.java  |   40 +--
 .../giraph/worker/VertexInputSplitsCallable.java   |   34 +--
 .../worker/VertexInputSplitsCallableFactory.java   |   22 +-
 .../giraph/worker/WorkerAggregatorHandler.java     |    4 +-
 .../org/apache/giraph/worker/WorkerContext.java    |   16 +-
 .../org/apache/giraph/comm/ConnectionTest.java     |   17 +-
 .../org/apache/giraph/comm/RequestFailureTest.java |   22 +-
 .../java/org/apache/giraph/comm/RequestTest.java   |   58 ++--
 .../org/apache/giraph/comm/SaslConnectionTest.java |   13 +-
 .../org/apache/giraph/comm/TestMessageStores.java  |   38 +--
 .../org/apache/giraph/conf/TestObjectCreation.java |   20 +-
 .../apache/giraph/graph/TestVertexAndEdges.java    |   37 +-
 .../TestAdjacencyListTextVertexOutputFormat.java   |   20 +-
 .../java/org/apache/giraph/io/TestEdgeInput.java   |   29 +-
 .../java/org/apache/giraph/io/TestFilters.java     |   12 +-
 .../giraph/io/TestIdWithValueTextOutputFormat.java |   17 +-
 .../org/apache/giraph/io/TestJsonBase64Format.java |   17 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |   33 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |   55 +--
 .../master/TestComputationCombinerTypes.java       |  160 ++++++++
 .../apache/giraph/master/TestMasterObserver.java   |   55 +++-
 .../apache/giraph/master/TestSwitchClasses.java    |  268 +++++++++++++
 .../partition/TestGiraphTransferRegulator.java     |   13 +-
 .../giraph/partition/TestPartitionStores.java      |  111 +++----
 .../apache/giraph/utils/ComputationCountEdges.java |   41 ++
 .../giraph/utils/IntIntNullNoOpComputation.java    |   30 ++
 .../apache/giraph/utils/IntNoOpComputation.java    |   29 ++
 .../apache/giraph/utils/LongNoOpComputation.java   |   29 ++
 .../java/org/apache/giraph/utils/MockUtils.java    |   87 ++---
 .../org/apache/giraph/utils/NoOpComputation.java   |   44 +++
 .../giraph/vertices/IntIntNullVertexDoNothing.java |   25 --
 .../apache/giraph/vertices/VertexCountEdges.java   |   33 --
 .../apache/giraph/vertices/VertexDoNothing.java    |   33 --
 .../java/org/apache/giraph/yarn/TestYarnJob.java   |   12 +-
 .../examples/AggregatorsTestComputation.java       |  137 +++++++
 .../giraph/examples/AggregatorsTestVertex.java     |  134 -------
 .../examples/ConnectedComponentsComputation.java   |  105 ++++++
 .../giraph/examples/ConnectedComponentsVertex.java |  101 -----
 .../giraph/examples/IdentityComputation.java       |   47 +++
 .../org/apache/giraph/examples/IdentityVertex.java |   45 ---
 .../examples/LongDoubleDoubleTextInputFormat.java  |   13 +-
 .../examples/LongDoubleNullTextInputFormat.java    |   13 +-
 ...NormalizingLongDoubleDoubleTextInputFormat.java |   15 +-
 .../giraph/examples/PageRankComputation.java       |   55 +++
 .../org/apache/giraph/examples/PageRankVertex.java |   54 ---
 .../examples/PartitionContextTestVertex.java       |  115 ------
 .../giraph/examples/RandomWalkComputation.java     |  173 +++++++++
 .../apache/giraph/examples/RandomWalkVertex.java   |  163 --------
 .../examples/RandomWalkVertexMasterCompute.java    |   18 +-
 .../examples/RandomWalkWithRestartComputation.java |   85 +++++
 .../examples/RandomWalkWithRestartVertex.java      |   80 ----
 .../giraph/examples/RandomWalkWorkerContext.java   |    8 +-
 .../apache/giraph/examples/SimpleCheckpoint.java   |  292 +++++++++++++++
 .../giraph/examples/SimpleCheckpointVertex.java    |  286 --------------
 .../giraph/examples/SimpleCombinerComputation.java |   70 ++++
 .../giraph/examples/SimpleCombinerVertex.java      |   65 ----
 .../giraph/examples/SimpleFailComputation.java     |   74 ++++
 .../apache/giraph/examples/SimpleFailVertex.java   |   69 ----
 .../examples/SimpleInDegreeCountComputation.java   |   58 +++
 .../giraph/examples/SimpleInDegreeCountVertex.java |   53 ---
 ...eLongDoubleDoubleDoubleIdentityComputation.java |   32 ++
 ...SimpleLongDoubleDoubleDoubleIdentityVertex.java |   32 --
 .../examples/SimpleMasterComputeComputation.java   |  112 ++++++
 .../giraph/examples/SimpleMasterComputeVertex.java |  108 ------
 .../giraph/examples/SimpleMsgComputation.java      |   67 ++++
 .../apache/giraph/examples/SimpleMsgVertex.java    |   62 ---
 .../examples/SimpleMutateGraphComputation.java     |  198 ++++++++++
 .../giraph/examples/SimpleMutateGraphVertex.java   |  197 ----------
 .../examples/SimpleOutDegreeCountComputation.java  |   46 +++
 .../examples/SimpleOutDegreeCountVertex.java       |   43 ---
 .../giraph/examples/SimplePageRankComputation.java |  250 ++++++++++++
 .../giraph/examples/SimplePageRankVertex.java      |  248 ------------
 .../examples/SimpleShortestPathsComputation.java   |   86 +++++
 .../giraph/examples/SimpleShortestPathsVertex.java |   81 ----
 .../examples/SimpleSuperstepComputation.java       |  152 ++++++++
 .../giraph/examples/SimpleSuperstepVertex.java     |  150 --------
 .../examples/SimpleTextVertexOutputFormat.java     |    2 +-
 .../examples/SimpleTriangleClosingComputation.java |  154 ++++++++
 .../examples/SimpleTriangleClosingVertex.java      |  151 --------
 .../examples/SimpleVertexWithWorkerContext.java    |   22 +-
 .../examples/TestComputationStateComputation.java  |  109 ++++++
 .../org/apache/giraph/examples/VerifyMessage.java  |   50 ++--
 ...xWithDoubleValueDoubleEdgeTextOutputFormat.java |    2 +-
 ...texWithDoubleValueNullEdgeTextOutputFormat.java |    2 +-
 .../java/org/apache/giraph/TestAutoCheckpoint.java |   16 +-
 .../test/java/org/apache/giraph/TestBspBasic.java  |   85 ++---
 .../org/apache/giraph/TestComputationState.java    |   65 ++++
 .../org/apache/giraph/TestGraphPartitioner.java    |   54 ++--
 .../org/apache/giraph/TestManualCheckpoint.java    |   28 +-
 .../java/org/apache/giraph/TestMaxSuperstep.java   |   18 +-
 .../java/org/apache/giraph/TestMutateGraph.java    |   10 +-
 .../org/apache/giraph/TestNotEnoughMapTasks.java   |   10 +-
 .../org/apache/giraph/TestPartitionContext.java    |   67 ----
 .../aggregators/TestAggregatorsHandling.java       |   26 +-
 .../ConnectedComponentsComputationTest.java        |  120 ++++++
 ...ConnectedComponentsComputationTestInMemory.java |  127 +++++++
 .../examples/ConnectedComponentsVertexTest.java    |  120 ------
 .../ConnectedComponentsVertexTestInMemory.java     |  130 -------
 .../giraph/examples/PageRankComputationTest.java   |   79 ++++
 .../apache/giraph/examples/PageRankVertexTest.java |   78 ----
 .../RandomWalkWithRestartComputationTest.java      |  111 ++++++
 .../examples/RandomWalkWithRestartVertexTest.java  |  109 ------
 .../SimpleShortestPathsComputationTest.java        |  164 ++++++++
 .../examples/SimpleShortestPathsVertexTest.java    |  161 --------
 .../SimpleTriangleClosingComputationTest.java      |   92 +++++
 .../examples/SimpleTriangleClosingVertexTest.java  |   93 -----
 .../org/apache/giraph/examples/TestPageRank.java   |   14 +-
 .../examples/TryMultiIpcBindingPortsTest.java      |    2 +-
 .../apache/giraph/vertex/TestComputationTypes.java |  244 ++++++++++++
 .../org/apache/giraph/vertex/TestVertexTypes.java  |  248 ------------
 .../io/hbase/TestHBaseRootMarkerVertextFormat.java |   14 +-
 .../io/hbase/edgemarker/TableEdgeInputFormat.java  |    4 +-
 .../io/hbase/edgemarker/TableEdgeOutputFormat.java |    2 +-
 .../giraph/io/hcatalog/HCatGiraphRunner.java       |   34 +-
 .../io/hcatalog/HCatalogVertexInputFormat.java     |    8 +-
 .../io/hcatalog/HCatalogVertexOutputFormat.java    |   10 +-
 .../org/apache/giraph/hive/HiveGiraphRunner.java   |   38 +-
 .../DefaultConfigurableAndTableSchemaAware.java    |    6 +-
 .../giraph/hive/input/edge/AbstractHiveToEdge.java |    4 +-
 .../hive/input/edge/HiveEdgeInputFormat.java       |    2 +-
 .../giraph/hive/input/edge/HiveEdgeReader.java     |    2 +-
 .../hive/input/vertex/AbstractHiveToVertex.java    |    4 +-
 .../giraph/hive/input/vertex/HiveToVertex.java     |    2 +-
 .../hive/input/vertex/HiveVertexInputFormat.java   |    2 +-
 .../giraph/hive/input/vertex/HiveVertexReader.java |    6 +-
 .../hive/input/vertex/SimpleHiveToVertex.java      |    8 +-
 .../giraph/hive/output/AbstractVertexToHive.java   |    2 +-
 .../giraph/hive/output/HiveVertexOutputFormat.java |    2 +-
 .../giraph/hive/output/HiveVertexWriter.java       |    2 +-
 .../giraph/hive/output/SimpleVertexToHive.java     |    4 +-
 .../apache/giraph/hive/output/VertexToHive.java    |    2 +-
 .../output/examples/HiveOutputIntIntVertex.java    |    2 +-
 .../hive/computations/ComputationCountEdges.java   |   36 ++
 .../hive/computations/ComputationSumEdges.java     |   42 ++
 .../giraph/hive/computations/package-info.java     |   22 ++
 .../giraph/hive/input/HiveEdgeInputTest.java       |   10 +-
 .../giraph/hive/input/HiveVertexInputTest.java     |   12 +-
 .../apache/giraph/hive/output/HiveOutputTest.java  |    4 +-
 .../giraph/hive/vertexes/VertexCountEdges.java     |   34 --
 .../giraph/hive/vertexes/VertexSumEdges.java       |   39 --
 .../apache/giraph/hive/vertexes/package-info.java  |   22 --
 299 files changed, 6685 insertions(+), 6079 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 52f8099..c20ded9 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,9 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-667: Decouple Vertex data and Computation, make Computation
+  and Combiner classes switchable (majakabiljo)
+
   GIRAPH-608: Spelling error in Combiner.java (Michael Aro via aching)
 
   GIRAPH-666: Netty execs threads and metrics threads don't get 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-accumulo/src/main/java/org/apache/giraph/io/accumulo/AccumuloVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-accumulo/src/main/java/org/apache/giraph/io/accumulo/AccumuloVertexInputFormat.java b/giraph-accumulo/src/main/java/org/apache/giraph/io/accumulo/AccumuloVertexInputFormat.java
index 221fabd..c286ed4 100644
--- a/giraph-accumulo/src/main/java/org/apache/giraph/io/accumulo/AccumuloVertexInputFormat.java
+++ b/giraph-accumulo/src/main/java/org/apache/giraph/io/accumulo/AccumuloVertexInputFormat.java
@@ -69,7 +69,7 @@ public abstract class AccumuloVertexInputFormat<
       extends VertexReader<I, V, E> {
 
     /** Giraph configuration */
-    private ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    private ImmutableClassesGiraphConfiguration<I, V, E>
     configuration;
     /**
      * Used by subclasses to read key/value pairs.
@@ -86,7 +86,7 @@ public abstract class AccumuloVertexInputFormat<
       this.reader = reader;
     }
 
-    public ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    public ImmutableClassesGiraphConfiguration<I, V, E>
     getConfiguration() {
       return configuration;
     }
@@ -105,7 +105,7 @@ public abstract class AccumuloVertexInputFormat<
       reader.initialize(inputSplit, context);
       this.context = context;
       this.configuration =
-          new ImmutableClassesGiraphConfiguration<I, V, E, Writable>(
+          new ImmutableClassesGiraphConfiguration<I, V, E>(
               context.getConfiguration());
     }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/TestAccumuloVertexFormat.java
----------------------------------------------------------------------
diff --git a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/TestAccumuloVertexFormat.java b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/TestAccumuloVertexFormat.java
index 6698c9a..0ee9666 100644
--- a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/TestAccumuloVertexFormat.java
+++ b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/TestAccumuloVertexFormat.java
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.giraph.BspCase;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.io.accumulo.edgemarker.AccumuloEdgeInputFormat;
 import org.apache.giraph.io.accumulo.edgemarker.AccumuloEdgeOutputFormat;
@@ -133,7 +134,7 @@ public class TestAccumuloVertexFormat extends BspCase{
         GiraphJob job = new GiraphJob(conf, getCallingMethodName());
         setupConfiguration(job);
         GiraphConfiguration giraphConf = job.getConfiguration();
-        giraphConf.setVertexClass(EdgeNotification.class);
+        giraphConf.setComputationClass(EdgeNotification.class);
         giraphConf.setVertexInputFormatClass(AccumuloEdgeInputFormat.class);
         giraphConf.setVertexOutputFormatClass(AccumuloEdgeOutputFormat.class);
 
@@ -170,16 +171,17 @@ public class TestAccumuloVertexFormat extends BspCase{
     The test set only has a 1-1 parent-to-child ratio for this unit test.
      */
     public static class EdgeNotification
-            extends Vertex<Text, Text, Text, Text> {
-        @Override
-        public void compute(Iterable<Text> messages) throws IOException {
+            extends BasicComputation<Text, Text, Text, Text> {
+      @Override
+      public void compute(Vertex<Text, Text, Text> vertex,
+          Iterable<Text> messages) throws IOException {
           for (Text message : messages) {
-            getValue().set(message);
+            vertex.getValue().set(message);
           }
           if(getSuperstep() == 0) {
-            sendMessageToAllEdges(getId());
+            sendMessageToAllEdges(vertex, vertex.getId());
           }
-          voteToHalt();
-        }
+        vertex.voteToHalt();
+      }
     }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
index 5855cfc..108ae61 100644
--- a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
+++ b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
@@ -73,11 +73,11 @@ public class AccumuloEdgeInputFormat
     /*
    Each Key/Value contains the information needed to construct the vertices.
      */
-    public Vertex<Text, Text, Text, ?> getCurrentVertex()
+    public Vertex<Text, Text, Text> getCurrentVertex()
         throws IOException, InterruptedException {
       Key key = getRecordReader().getCurrentKey();
       Value value = getRecordReader().getCurrentValue();
-      Vertex<Text, Text, Text, ?> vertex =
+      Vertex<Text, Text, Text> vertex =
           getConfiguration().createVertex();
       Text vertexId = key.getRow();
       List<Edge<Text, Text>> edges = Lists.newLinkedList();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeOutputFormat.java b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeOutputFormat.java
index f999497..c2ebbe2 100644
--- a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeOutputFormat.java
+++ b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeOutputFormat.java
@@ -64,7 +64,7 @@ public class AccumuloEdgeOutputFormat
      Write back a mutation that adds a qualifier for 'parent' containing the vertex value
      as the cell value. Assume the vertex ID corresponds to a key.
      */
-    public void writeVertex(Vertex<Text, Text, Text, ?> vertex)
+    public void writeVertex(Vertex<Text, Text, Text> vertex)
         throws IOException, InterruptedException {
       RecordWriter<Text, Mutation> writer = getRecordWriter();
       Mutation mt = new Mutation(vertex.getId());

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/AggregatorsBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/AggregatorsBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/AggregatorsBenchmark.java
index 12304bb..539bd7d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/AggregatorsBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/AggregatorsBenchmark.java
@@ -20,6 +20,7 @@ package org.apache.giraph.benchmark;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.io.formats.PseudoRandomInputFormatConstants;
@@ -52,10 +53,13 @@ public class AggregatorsBenchmark extends GiraphBenchmark {
   /**
    * Vertex class for AggregatorsBenchmark
    */
-  public static class AggregatorsBenchmarkVertex extends
-      Vertex<LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> {
+  public static class AggregatorsBenchmarkComputation extends
+      BasicComputation<LongWritable, DoubleWritable, DoubleWritable,
+          DoubleWritable> {
     @Override
-    public void compute(Iterable<DoubleWritable> messages) throws IOException {
+    public void compute(
+        Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex,
+        Iterable<DoubleWritable> messages) throws IOException {
       int n = getNumAggregators(getConf());
       long superstep = getSuperstep();
       int w = getWorkerContextAggregated(getConf(), superstep);
@@ -71,7 +75,7 @@ public class AggregatorsBenchmark extends GiraphBenchmark {
             ((LongWritable) getAggregatedValue("p" + i)).get());
       }
       if (superstep > 2) {
-        voteToHalt();
+        vertex.voteToHalt();
       }
     }
   }
@@ -201,7 +205,7 @@ public class AggregatorsBenchmark extends GiraphBenchmark {
   @Override
   protected void prepareConfiguration(GiraphConfiguration conf,
       CommandLine cmd) {
-    conf.setVertexClass(AggregatorsBenchmarkVertex.class);
+    conf.setComputationClass(AggregatorsBenchmarkComputation.class);
     conf.setMasterComputeClass(AggregatorsBenchmarkMasterCompute.class);
     conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);
     conf.setWorkerContextClass(AggregatorsBenchmarkWorkerContext.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
index 0f8d284..bd2939e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
@@ -31,7 +31,7 @@ import com.google.common.collect.Sets;
 import java.util.Set;
 
 /**
- * Benchmark for {@link PageRankVertex}
+ * Benchmark for {@link PageRankComputation}
  */
 public class PageRankBenchmark extends GiraphBenchmark {
   @Override
@@ -44,7 +44,7 @@ public class PageRankBenchmark extends GiraphBenchmark {
   @Override
   protected void prepareConfiguration(GiraphConfiguration conf,
       CommandLine cmd) {
-    conf.setVertexClass(PageRankVertex.class);
+    conf.setComputationClass(PageRankComputation.class);
     conf.setOutEdgesClass(IntNullArrayEdges.class);
     conf.setCombinerClass(FloatSumCombiner.class);
     conf.setVertexInputFormatClass(
@@ -54,7 +54,7 @@ public class PageRankBenchmark extends GiraphBenchmark {
         BenchmarkOption.VERTICES.getOptionIntValue(cmd));
     conf.setInt(PseudoRandomInputFormatConstants.EDGES_PER_VERTEX,
         BenchmarkOption.EDGES_PER_VERTEX.getOptionIntValue(cmd));
-    conf.setInt(PageRankVertex.SUPERSTEP_COUNT,
+    conf.setInt(PageRankComputation.SUPERSTEP_COUNT,
         BenchmarkOption.SUPERSTEPS.getOptionIntValue(cmd));
     conf.setFloat(PseudoRandomInputFormatConstants.LOCAL_EDGES_MIN_RATIO,
         BenchmarkOption.LOCAL_EDGES_MIN_RATIO.getOptionFloatValue(cmd,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java
new file mode 100644
index 0000000..e891ff2
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankComputation.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.benchmark;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import java.io.IOException;
+
+/**
+ * Implementation of PageRank in which vertex ids are ints, page rank values
+ * are floats, and graph is unweighted.
+ */
+public class PageRankComputation extends BasicComputation<IntWritable,
+    FloatWritable, NullWritable, FloatWritable> {
+  /** Number of supersteps */
+  public static final String SUPERSTEP_COUNT =
+      "giraph.pageRank.superstepCount";
+
+  @Override
+  public void compute(
+      Vertex<IntWritable, FloatWritable, NullWritable> vertex,
+      Iterable<FloatWritable> messages) throws IOException {
+    if (getSuperstep() >= 1) {
+      float sum = 0;
+      for (FloatWritable message : messages) {
+        sum += message.get();
+      }
+      vertex.getValue().set((0.15f / getTotalNumVertices()) + 0.85f * sum);
+    }
+
+    if (getSuperstep() < getConf().getInt(SUPERSTEP_COUNT, 0)) {
+      sendMessageToAllEdges(vertex,
+          new FloatWritable(vertex.getValue().get() / vertex.getNumEdges()));
+    } else {
+      vertex.voteToHalt();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankVertex.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankVertex.java
deleted file mode 100644
index 9900a44..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankVertex.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.benchmark;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.NullWritable;
-
-import java.io.IOException;
-
-/**
- * Implementation of PageRank in which vertex ids are ints, page rank values
- * are floats, and graph is unweighted.
- */
-public class PageRankVertex extends Vertex<IntWritable, FloatWritable,
-    NullWritable, FloatWritable> {
-  /** Number of supersteps */
-  public static final String SUPERSTEP_COUNT =
-      "giraph.pageRank.superstepCount";
-
-  @Override
-  public void compute(Iterable<FloatWritable> messages) throws IOException {
-    if (getSuperstep() >= 1) {
-      float sum = 0;
-      for (FloatWritable message : messages) {
-        sum += message.get();
-      }
-      getValue().set((0.15f / getTotalNumVertices()) + 0.85f * sum);
-    }
-
-    if (getSuperstep() < getConf().getInt(SUPERSTEP_COUNT, 0)) {
-      sendMessageToAllEdges(
-          new FloatWritable(getValue().get() / getNumEdges()));
-    } else {
-      voteToHalt();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
index 5c7e019..cf2e6eb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
@@ -20,6 +20,7 @@ package org.apache.giraph.benchmark;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.io.formats.PseudoRandomInputFormatConstants;
@@ -35,6 +36,7 @@ import org.apache.log4j.Logger;
 
 import com.google.common.collect.Sets;
 
+import java.io.IOException;
 import java.util.Random;
 import java.util.Set;
 
@@ -257,25 +259,26 @@ public class RandomMessageBenchmark extends GiraphBenchmark {
   /**
    * Actual message computation (messaging in this case)
    */
-  public static class RandomMessageVertex extends Vertex<LongWritable,
-      DoubleWritable, DoubleWritable, BytesWritable> {
+  public static class RandomMessageComputation extends BasicComputation<
+      LongWritable, DoubleWritable, DoubleWritable, BytesWritable> {
     @Override
-    public void compute(Iterable<BytesWritable> messages) {
-      RandomMessageBenchmarkWorkerContext workerContext =
-          (RandomMessageBenchmarkWorkerContext) getWorkerContext();
+    public void compute(
+        Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex,
+        Iterable<BytesWritable> messages) throws IOException {
+      RandomMessageBenchmarkWorkerContext workerContext = getWorkerContext();
       if (getSuperstep() < workerContext.getNumSupersteps()) {
         for (int i = 0; i < workerContext.getNumMessagePerEdge(); i++) {
           workerContext.randomizeMessageBytes();
-          sendMessageToAllEdges(
+          sendMessageToAllEdges(vertex,
               new BytesWritable(workerContext.getMessageBytes()));
           long bytesSent = workerContext.getMessageBytes().length *
-              getNumEdges();
+              vertex.getNumEdges();
           aggregate(AGG_SUPERSTEP_TOTAL_BYTES, new LongWritable(bytesSent));
           aggregate(AGG_SUPERSTEP_TOTAL_MESSAGES,
-              new LongWritable(getNumEdges()));
+              new LongWritable(vertex.getNumEdges()));
         }
       } else {
-        voteToHalt();
+        vertex.voteToHalt();
       }
     }
   }
@@ -290,7 +293,7 @@ public class RandomMessageBenchmark extends GiraphBenchmark {
   @Override
   protected void prepareConfiguration(GiraphConfiguration conf,
       CommandLine cmd) {
-    conf.setVertexClass(RandomMessageVertex.class);
+    conf.setComputationClass(RandomMessageComputation.class);
     conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);
     conf.setWorkerContextClass(RandomMessageBenchmarkWorkerContext.class);
     conf.setMasterComputeClass(RandomMessageBenchmarkMasterCompute.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
index 8e6c877..0dd4529 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
@@ -58,13 +58,13 @@ public class ShortestPathsBenchmark extends GiraphBenchmark {
   @Override
   protected void prepareConfiguration(GiraphConfiguration conf,
       CommandLine cmd) {
-    conf.setVertexClass(ShortestPathsVertex.class);
+    conf.setComputationClass(ShortestPathsComputation.class);
     if (EDGES_CLASS.getOptionIntValue(cmd, 1) == 1) {
       conf.setOutEdgesClass(ArrayListEdges.class);
     } else {
       conf.setOutEdgesClass(HashMapEdges.class);
     }
-    LOG.info("Using class " + GiraphConstants.VERTEX_CLASS.get(conf));
+    LOG.info("Using class " + GiraphConstants.COMPUTATION_CLASS.get(conf));
     conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);
     if (!NO_COMBINER.optionTurnedOn(cmd)) {
       conf.setCombinerClass(MinimumDoubleCombiner.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsComputation.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsComputation.java
new file mode 100644
index 0000000..ad0600c
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsComputation.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.benchmark;
+
+import org.apache.giraph.graph.BasicComputation;
+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 java.io.IOException;
+
+/**
+ * Shortest paths algorithm.
+ */
+public class ShortestPathsComputation extends BasicComputation<LongWritable,
+    DoubleWritable, DoubleWritable, DoubleWritable> {
+  /** Source id. */
+  public static final String SOURCE_ID =
+      "giraph.shortestPathsBenchmark.sourceId";
+  /** Default source id. */
+  public static final long SOURCE_ID_DEFAULT = 1;
+
+  /**
+   * Check if vertex is source from which to calculate shortest paths.
+   *
+   * @param vertex Vertex
+   * @return True iff vertex is source for shortest paths
+   */
+  private boolean isSource(
+      Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex) {
+    return vertex.getId().get() ==
+        getConf().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+  }
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    if (getSuperstep() == 0) {
+      vertex.setValue(new DoubleWritable(Double.MAX_VALUE));
+    }
+
+    double minDist = isSource(vertex) ? 0d : Double.MAX_VALUE;
+    for (DoubleWritable message : messages) {
+      minDist = Math.min(minDist, message.get());
+    }
+
+    if (minDist < vertex.getValue().get()) {
+      vertex.setValue(new DoubleWritable(minDist));
+      for (Edge<LongWritable, DoubleWritable> edge : vertex.getEdges()) {
+        double distance = minDist + edge.getValue().get();
+        sendMessage(edge.getTargetVertexId(),
+            new DoubleWritable(distance));
+      }
+    }
+
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsVertex.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsVertex.java
deleted file mode 100644
index c1b77d1..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsVertex.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.benchmark;
-
-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 java.io.IOException;
-
-/**
- * Shortest paths algorithm.
- */
-public class ShortestPathsVertex extends Vertex<LongWritable, DoubleWritable,
-    DoubleWritable, DoubleWritable> {
-  /** Source id. */
-  public static final String SOURCE_ID =
-      "giraph.shortestPathsBenchmark.sourceId";
-  /** Default source id. */
-  public static final long SOURCE_ID_DEFAULT = 1;
-
-  private boolean isSource() {
-    return getId().get() == getConf().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
-  }
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) throws IOException {
-    if (getSuperstep() == 0) {
-      setValue(new DoubleWritable(Double.MAX_VALUE));
-    }
-
-    double minDist = isSource() ? 0d : Double.MAX_VALUE;
-    for (DoubleWritable message : messages) {
-      minDist = Math.min(minDist, message.get());
-    }
-
-    if (minDist < getValue().get()) {
-      setValue(new DoubleWritable(minDist));
-      for (Edge<LongWritable, DoubleWritable> edge : getEdges()) {
-        double distance = minDist + edge.getValue().get();
-        sendMessage(edge.getTargetVertexId(),
-            new DoubleWritable(distance));
-      }
-    }
-
-    voteToHalt();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
index 3fc514a..2077674 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
@@ -38,7 +38,7 @@ import com.google.common.collect.Sets;
 import java.util.Set;
 
 /**
- * Benchmark for {@link WeightedPageRankVertex}
+ * Benchmark for {@link WeightedPageRankComputation}
  */
 public class WeightedPageRankBenchmark extends GiraphBenchmark {
   /** Class logger */
@@ -88,7 +88,7 @@ public class WeightedPageRankBenchmark extends GiraphBenchmark {
    */
   protected void prepareConfiguration(GiraphConfiguration configuration,
       CommandLine cmd) {
-    configuration.setVertexClass(WeightedPageRankVertex.class);
+    configuration.setComputationClass(WeightedPageRankComputation.class);
     int edgesClassOption = EDGES_CLASS.getOptionIntValue(cmd, 1);
     switch (edgesClassOption) {
     case 0:
@@ -149,7 +149,7 @@ public class WeightedPageRankBenchmark extends GiraphBenchmark {
           SimpleLongRangePartitionerFactory.class);
     }
 
-    configuration.setInt(WeightedPageRankVertex.SUPERSTEP_COUNT,
+    configuration.setInt(WeightedPageRankComputation.SUPERSTEP_COUNT,
         BenchmarkOption.SUPERSTEPS.getOptionIntValue(cmd));
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankComputation.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankComputation.java
new file mode 100644
index 0000000..18182ed
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankComputation.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.benchmark;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.MutableEdge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+
+import java.io.IOException;
+
+/**
+ * Implementation of Page Rank algorithm on a weighted graph.
+ */
+public class WeightedPageRankComputation extends BasicComputation<LongWritable,
+    DoubleWritable, DoubleWritable, DoubleWritable> {
+  /** Number of supersteps */
+  public static final String SUPERSTEP_COUNT =
+      "giraph.weightedPageRank.superstepCount";
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    if (getSuperstep() == 0) {
+      // Normalize out edge weights
+      double outEdgeSum = 0;
+      for (Edge<LongWritable, DoubleWritable> edge : vertex.getEdges()) {
+        outEdgeSum += edge.getValue().get();
+      }
+      for (MutableEdge<LongWritable, DoubleWritable> edge :
+          vertex.getMutableEdges()) {
+        edge.setValue(new DoubleWritable(edge.getValue().get() / outEdgeSum));
+      }
+    } else {
+      double messageSum = 0;
+      for (DoubleWritable message : messages) {
+        messageSum += message.get();
+      }
+      vertex.getValue().set(
+          (0.15f / getTotalNumVertices()) + 0.85f * messageSum);
+    }
+
+    if (getSuperstep() < getConf().getInt(SUPERSTEP_COUNT, 0)) {
+      for (Edge<LongWritable, DoubleWritable> edge : vertex.getEdges()) {
+        sendMessage(edge.getTargetVertexId(),
+            new DoubleWritable(
+                vertex.getValue().get() * edge.getValue().get()));
+      }
+    } else {
+      vertex.voteToHalt();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankVertex.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankVertex.java
deleted file mode 100644
index 70f0f61..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankVertex.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.benchmark;
-
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.MutableEdge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-
-import java.io.IOException;
-
-/**
- * Implementation of Page Rank algorithm on a weighted graph.
- */
-public class WeightedPageRankVertex extends Vertex<LongWritable, DoubleWritable,
-    DoubleWritable, DoubleWritable> {
-  /** Number of supersteps */
-  public static final String SUPERSTEP_COUNT =
-      "giraph.weightedPageRank.superstepCount";
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) throws IOException {
-    if (getSuperstep() == 0) {
-      // Normalize out edge weights
-      double outEdgeSum = 0;
-      for (Edge<LongWritable, DoubleWritable> edge : getEdges()) {
-        outEdgeSum += edge.getValue().get();
-      }
-      for (MutableEdge<LongWritable, DoubleWritable> edge : getMutableEdges()) {
-        edge.setValue(new DoubleWritable(edge.getValue().get() / outEdgeSum));
-      }
-    } else {
-      double messageSum = 0;
-      for (DoubleWritable message : messages) {
-        messageSum += message.get();
-      }
-      getValue().set((0.15f / getTotalNumVertices()) + 0.85f * messageSum);
-    }
-
-    if (getSuperstep() < getConf().getInt(SUPERSTEP_COUNT, 0)) {
-      for (Edge<LongWritable, DoubleWritable> edge : getEdges()) {
-        sendMessage(edge.getTargetVertexId(),
-            new DoubleWritable(getValue().get() * edge.getValue().get()));
-      }
-    } else {
-      voteToHalt();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java b/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
index 178c96f..ff3f06d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java
@@ -60,12 +60,11 @@ import static org.apache.giraph.conf.GiraphConstants.CHECKPOINT_DIRECTORY;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public abstract class BspService<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements Watcher, CentralizedService<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    implements Watcher, CentralizedService<I, V, E> {
   /** Unset superstep */
   public static final long UNSET_SUPERSTEP = Long.MIN_VALUE;
   /** Input superstep (superstep when loading the vertices happens) */
@@ -218,7 +217,7 @@ public abstract class BspService<I extends WritableComparable,
   private final List<BspEvent> registeredBspEvents =
       new ArrayList<BspEvent>();
   /** Immutable configuration of the job*/
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Job context (mainly for progress) */
   private final Mapper<?, ?, ?, ?>.Context context;
   /** Cached superstep (from ZooKeeper) */
@@ -236,9 +235,9 @@ public abstract class BspService<I extends WritableComparable,
   /** Combination of hostname '_' partition (unique id) */
   private final String hostnamePartitionId;
   /** Graph partitioner */
-  private final GraphPartitionerFactory<I, V, E, M> graphPartitionerFactory;
+  private final GraphPartitionerFactory<I, V, E> graphPartitionerFactory;
   /** Mapper that will do the graph computation */
-  private final GraphTaskManager<I, V, E, M> graphTaskManager;
+  private final GraphTaskManager<I, V, E> graphTaskManager;
   /** File system */
   private final FileSystem fs;
   /** Checkpoint frequency */
@@ -255,7 +254,7 @@ public abstract class BspService<I extends WritableComparable,
   public BspService(String serverPortList,
       int sessionMsecTimeout,
       Mapper<?, ?, ?, ?>.Context context,
-      GraphTaskManager<I, V, E, M> graphTaskManager) {
+      GraphTaskManager<I, V, E> graphTaskManager) {
     this.vertexInputSplitsEvents = new InputSplitEvents(context);
     this.edgeInputSplitsEvents = new InputSplitEvents(context);
     this.connectedEvent = new PredicateLock(context);
@@ -280,8 +279,7 @@ public abstract class BspService<I extends WritableComparable,
 
     this.context = context;
     this.graphTaskManager = graphTaskManager;
-    this.conf = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
-        context.getConfiguration());
+    this.conf = graphTaskManager.getConf();
     this.jobId = conf.get("mapred.job.id", "Unknown Job");
     this.taskPartition = conf.getTaskPartition();
     this.restartedSuperstep = conf.getLong(
@@ -605,7 +603,7 @@ public abstract class BspService<I extends WritableComparable,
     return fs;
   }
 
-  public final ImmutableClassesGiraphConfiguration<I, V, E, M>
+  public final ImmutableClassesGiraphConfiguration<I, V, E>
   getConfiguration() {
     return conf;
   }
@@ -626,7 +624,7 @@ public abstract class BspService<I extends WritableComparable,
     return taskPartition;
   }
 
-  public final GraphTaskManager<I, V, E, M> getGraphTaskManager() {
+  public final GraphTaskManager<I, V, E> getGraphTaskManager() {
     return graphTaskManager;
   }
 
@@ -885,7 +883,7 @@ public abstract class BspService<I extends WritableComparable,
    *
    * @return Instantiated graph partitioner factory
    */
-  protected GraphPartitionerFactory<I, V, E, M> getGraphPartitionerFactory() {
+  protected GraphPartitionerFactory<I, V, E> getGraphPartitionerFactory() {
     return graphPartitionerFactory;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
index 2281903..ff3e427 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java
@@ -30,11 +30,10 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface CentralizedService<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
 
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
index 5f84ece..fb98b00 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java
@@ -33,12 +33,11 @@ import java.io.IOException;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface CentralizedServiceMaster<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> extends
-    CentralizedService<I, V, E, M> {
+    V extends Writable, E extends Writable> extends
+    CentralizedService<I, V, E> {
   /**
    * Setup (must be called prior to any other function)
    */

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
index 1c7bde4..4b0f985 100644
--- a/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceWorker.java
@@ -21,7 +21,6 @@ package org.apache.giraph.bsp;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.WorkerClient;
 import org.apache.giraph.graph.FinishedSuperstepStats;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.graph.VertexEdgeCount;
 import org.apache.giraph.io.superstep_output.SuperstepOutput;
@@ -47,12 +46,11 @@ import java.util.List;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface CentralizedServiceWorker<I extends WritableComparable,
-  V extends Writable, E extends Writable, M extends Writable>
-  extends CentralizedService<I, V, E, M> {
+  V extends Writable, E extends Writable>
+  extends CentralizedService<I, V, E> {
   /**
    * Setup (must be called prior to any other function)
    *
@@ -73,7 +71,7 @@ public interface CentralizedServiceWorker<I extends WritableComparable,
    *
    * @return Worker client
    */
-  WorkerClient<I, V, E, M> getWorkerClient();
+  WorkerClient<I, V, E> getWorkerClient();
 
   /**
    * Get the worker context.
@@ -97,7 +95,7 @@ public interface CentralizedServiceWorker<I extends WritableComparable,
    *
    * @return The partition store for this worker.
    */
-  PartitionStore<I, V, E, M> getPartitionStore();
+  PartitionStore<I, V, E> getPartitionStore();
 
   /**
    *  Both the vertices and the messages need to be checkpointed in order
@@ -121,23 +119,19 @@ public interface CentralizedServiceWorker<I extends WritableComparable,
    * Take all steps prior to actually beginning the computation of a
    * superstep.
    *
-   * @param graphState Current graph state
    * @return Collection of all the partition owners from the master for this
    *         superstep.
    */
-  Collection<? extends PartitionOwner> startSuperstep(
-      GraphState<I, V, E, M> graphState);
+  Collection<? extends PartitionOwner> startSuperstep();
 
   /**
    * Worker is done with its portion of the superstep.  Report the
    * worker level statistics after the computation.
    *
-   * @param graphState Current graph state
    * @param partitionStatsList All the partition stats for this worker
    * @return Stats of the superstep completion
    */
   FinishedSuperstepStats finishSuperstep(
-      GraphState<I, V, E, M> graphState,
       List<PartitionStats> partitionStatsList);
 
   /**
@@ -195,7 +189,7 @@ public interface CentralizedServiceWorker<I extends WritableComparable,
    *
    * @return the GraphTaskManager instance for this compute node
    */
-  GraphTaskManager<I, V, E, M> getGraphTaskManager();
+  GraphTaskManager<I, V, E> getGraphTaskManager();
 
   /**
    * Operations that will be called if there is a failure by a worker.
@@ -207,7 +201,7 @@ public interface CentralizedServiceWorker<I extends WritableComparable,
    *
    * @return Server data
    */
-  ServerData<I, V, E, M> getServerData();
+  ServerData<I, V, E> getServerData();
 
   /**
    * Get worker aggregator handler

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/SendCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendCache.java
index 1e8bdf9..92d0926 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendCache.java
@@ -64,7 +64,7 @@ public abstract class SendCache<I extends WritableComparable, T,
    *                              ratio of the average request size)
    */
   public SendCache(ImmutableClassesGiraphConfiguration conf,
-                   CentralizedServiceWorker<?, ?, ?, ?> serviceWorker,
+                   CentralizedServiceWorker<?, ?, ?> serviceWorker,
                    int maxRequestSize,
                    float additionalRequestSize) {
     this.conf = conf;
@@ -176,4 +176,8 @@ public abstract class SendCache<I extends WritableComparable, T,
     }
     return allData;
   }
+
+  public ImmutableClassesGiraphConfiguration getConf() {
+    return conf;
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/SendEdgeCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendEdgeCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendEdgeCache.java
index fbc911f..5513da2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendEdgeCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendEdgeCache.java
@@ -46,7 +46,7 @@ public class SendEdgeCache<I extends WritableComparable, E extends Writable>
    * @param serviceWorker Service worker
    */
   public SendEdgeCache(ImmutableClassesGiraphConfiguration conf,
-                       CentralizedServiceWorker<?, ?, ?, ?> serviceWorker) {
+                       CentralizedServiceWorker<?, ?, ?> serviceWorker) {
     super(conf, serviceWorker, MAX_EDGE_REQUEST_SIZE.get(conf),
         ADDITIONAL_EDGE_REQUEST_SIZE.get(conf));
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageCache.java
index 7d2a888..40023c2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendMessageCache.java
@@ -45,14 +45,15 @@ public class SendMessageCache<I extends WritableComparable, M extends Writable>
    * @param serviceWorker Service worker
    */
   public SendMessageCache(ImmutableClassesGiraphConfiguration conf,
-      CentralizedServiceWorker<?, ?, ?, ?> serviceWorker) {
+      CentralizedServiceWorker<?, ?, ?> serviceWorker) {
     super(conf, serviceWorker, MAX_MSG_REQUEST_SIZE.get(conf),
         ADDITIONAL_MSG_REQUEST_SIZE.get(conf));
   }
 
   @Override
   public ByteArrayVertexIdMessages<I, M> createByteArrayVertexIdData() {
-    return new ByteArrayVertexIdMessages<I, M>();
+    return new ByteArrayVertexIdMessages<I, M>(
+        getConf().getOutgoingMessageValueClass());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
index 67f74f1..9348e61 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
@@ -33,14 +33,13 @@ import java.util.Map;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class SendMutationsCache<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /** Internal cache */
-  private Map<Integer, Map<I, VertexMutations<I, V, E, M>>> mutationCache =
-      new HashMap<Integer, Map<I, VertexMutations<I, V, E, M>>>();
+  private Map<Integer, Map<I, VertexMutations<I, V, E>>> mutationCache =
+      new HashMap<Integer, Map<I, VertexMutations<I, V, E>>>();
   /** Number of mutations in each partition */
   private final Map<Integer, Integer> mutationCountMap =
       new HashMap<Integer, Integer>();
@@ -53,17 +52,17 @@ public class SendMutationsCache<I extends WritableComparable,
    * @param destVertexId Destination vertex id
    * @return Mutations for the vertex
    */
-  private VertexMutations<I, V, E, M> getVertexMutations(
+  private VertexMutations<I, V, E> getVertexMutations(
       Integer partitionId, I destVertexId) {
-    Map<I, VertexMutations<I, V, E, M>> idMutations =
+    Map<I, VertexMutations<I, V, E>> idMutations =
         mutationCache.get(partitionId);
     if (idMutations == null) {
-      idMutations = new HashMap<I, VertexMutations<I, V, E, M>>();
+      idMutations = new HashMap<I, VertexMutations<I, V, E>>();
       mutationCache.put(partitionId, idMutations);
     }
-    VertexMutations<I, V, E, M> mutations = idMutations.get(destVertexId);
+    VertexMutations<I, V, E> mutations = idMutations.get(destVertexId);
     if (mutations == null) {
-      mutations = new VertexMutations<I, V, E, M>();
+      mutations = new VertexMutations<I, V, E>();
       idMutations.put(destVertexId, mutations);
     }
     return mutations;
@@ -97,7 +96,7 @@ public class SendMutationsCache<I extends WritableComparable,
   public int addEdgeMutation(
       Integer partitionId, I destVertexId, Edge<I, E> edge) {
     // Get the mutations for this partition
-    VertexMutations<I, V, E, M> mutations =
+    VertexMutations<I, V, E> mutations =
         getVertexMutations(partitionId, destVertexId);
 
     // Add the edge
@@ -118,7 +117,7 @@ public class SendMutationsCache<I extends WritableComparable,
   public int removeEdgeMutation(
       Integer partitionId, I vertexIndex, I destinationVertexIndex) {
     // Get the mutations for this partition
-    VertexMutations<I, V, E, M> mutations =
+    VertexMutations<I, V, E> mutations =
         getVertexMutations(partitionId, vertexIndex);
 
     // Remove the edge
@@ -136,9 +135,9 @@ public class SendMutationsCache<I extends WritableComparable,
    * @return Number of mutations in the partition.
    */
   public int addVertexMutation(
-      Integer partitionId, Vertex<I, V, E, M> vertex) {
+      Integer partitionId, Vertex<I, V, E> vertex) {
     // Get the mutations for this partition
-    VertexMutations<I, V, E, M> mutations =
+    VertexMutations<I, V, E> mutations =
         getVertexMutations(partitionId, vertex.getId());
 
     // Add the vertex
@@ -158,7 +157,7 @@ public class SendMutationsCache<I extends WritableComparable,
   public int removeVertexMutation(
       Integer partitionId, I destVertexId) {
     // Get the mutations for this partition
-    VertexMutations<I, V, E, M> mutations =
+    VertexMutations<I, V, E> mutations =
         getVertexMutations(partitionId, destVertexId);
 
     // Remove the vertex
@@ -174,9 +173,9 @@ public class SendMutationsCache<I extends WritableComparable,
    * @param partitionId Partition id
    * @return Removed partition mutations
    */
-  public Map<I, VertexMutations<I, V, E, M>> removePartitionMutations(
+  public Map<I, VertexMutations<I, V, E>> removePartitionMutations(
       int partitionId) {
-    Map<I, VertexMutations<I, V, E, M>> idMutations =
+    Map<I, VertexMutations<I, V, E>> idMutations =
         mutationCache.remove(partitionId);
     mutationCountMap.put(partitionId, 0);
     return idMutations;
@@ -187,12 +186,12 @@ public class SendMutationsCache<I extends WritableComparable,
    *
    * @return All vertex mutations for all partitions
    */
-  public Map<Integer, Map<I, VertexMutations<I, V, E, M>>>
+  public Map<Integer, Map<I, VertexMutations<I, V, E>>>
   removeAllPartitionMutations() {
-    Map<Integer, Map<I, VertexMutations<I, V, E, M>>> allMutations =
+    Map<Integer, Map<I, VertexMutations<I, V, E>>> allMutations =
         mutationCache;
     mutationCache =
-        new HashMap<Integer, Map<I, VertexMutations<I, V, E, M>>>();
+        new HashMap<Integer, Map<I, VertexMutations<I, V, E>>>();
     mutationCountMap.clear();
     return allMutations;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
index 31cf052..68ce095 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
@@ -36,23 +36,19 @@ import org.apache.log4j.Logger;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class SendPartitionCache<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(SendPartitionCache.class);
   /** Input split vertex cache (only used when loading from input split) */
-  private final Map<PartitionOwner, Partition<I, V, E, M>>
+  private final Map<PartitionOwner, Partition<I, V, E>>
   ownerPartitionMap = Maps.newHashMap();
-  /** Number of messages in each partition */
-  private final Map<PartitionOwner, Integer> messageCountMap =
-      Maps.newHashMap();
   /** Context */
   private final Mapper<?, ?, ?, ?>.Context context;
   /** Configuration */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /**
    *  Regulates the size of outgoing Collections of vertices read
    * by the local worker during INPUT_SUPERSTEP that are to be
@@ -69,7 +65,7 @@ public class SendPartitionCache<I extends WritableComparable,
    */
   public SendPartitionCache(
       Mapper<?, ?, ?, ?>.Context context,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration) {
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration) {
     this.context = context;
     this.configuration = configuration;
     transferRegulator =
@@ -89,9 +85,9 @@ public class SendPartitionCache<I extends WritableComparable,
    * @param vertex Vertex to add
    * @return A partition to send or null, if requirements are not met
    */
-  public Partition<I, V, E, M> addVertex(PartitionOwner partitionOwner,
-                                         Vertex<I, V, E, M> vertex) {
-    Partition<I, V, E, M> partition =
+  public Partition<I, V, E> addVertex(PartitionOwner partitionOwner,
+                                         Vertex<I, V, E> vertex) {
+    Partition<I, V, E> partition =
         ownerPartitionMap.get(partitionOwner);
     if (partition == null) {
       partition = configuration.createPartition(
@@ -101,7 +97,7 @@ public class SendPartitionCache<I extends WritableComparable,
     }
     transferRegulator.incrementCounters(partitionOwner, vertex);
 
-    Vertex<I, V, E, M> oldVertex =
+    Vertex<I, V, E> oldVertex =
         partition.putVertex(vertex);
     if (oldVertex != null) {
       LOG.warn("addVertex: Replacing vertex " + oldVertex +
@@ -121,7 +117,7 @@ public class SendPartitionCache<I extends WritableComparable,
    *
    * @return Owner partition map
    */
-  public Map<PartitionOwner, Partition<I, V, E, M>> getOwnerPartitionMap() {
+  public Map<PartitionOwner, Partition<I, V, E>> getOwnerPartitionMap() {
     return ownerPartitionMap;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java b/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
index 743a6f8..788be53 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java
@@ -43,34 +43,38 @@ import java.util.concurrent.ConcurrentHashMap;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class ServerData<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
+  /** Configuration */
+  private final ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Partition store for this worker. */
-  private volatile PartitionStore<I, V, E, M> partitionStore;
+  private volatile PartitionStore<I, V, E> partitionStore;
   /** Edge store for this worker. */
-  private final EdgeStore<I, V, E, M> edgeStore;
+  private final EdgeStore<I, V, E> edgeStore;
   /** Message store factory */
   private final
-  MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> messageStoreFactory;
+  MessageStoreFactory<I, Writable, MessageStoreByPartition<I, Writable>>
+  messageStoreFactory;
   /**
    * Message store for incoming messages (messages which will be consumed
    * in the next super step)
    */
-  private volatile MessageStoreByPartition<I, M> incomingMessageStore;
+  private volatile MessageStoreByPartition<I, Writable>
+  incomingMessageStore;
   /**
    * Message store for current messages (messages which we received in
    * previous super step and which will be consumed in current super step)
    */
-  private volatile MessageStoreByPartition<I, M> currentMessageStore;
+  private volatile MessageStoreByPartition<I, Writable>
+  currentMessageStore;
   /**
    * Map of partition ids to incoming vertex mutations from other workers.
    * (Synchronized access to values)
    */
-  private final ConcurrentHashMap<I, VertexMutations<I, V, E, M>>
-  vertexMutations = new ConcurrentHashMap<I, VertexMutations<I, V, E, M>>();
+  private final ConcurrentHashMap<I, VertexMutations<I, V, E>>
+  vertexMutations = new ConcurrentHashMap<I, VertexMutations<I, V, E>>();
   /**
    * Holds aggregtors which current worker owns from current superstep
    */
@@ -89,28 +93,30 @@ public class ServerData<I extends WritableComparable,
    * @param context Mapper context
    */
   public ServerData(
-      CentralizedServiceWorker<I, V, E, M> service,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf,
-      MessageStoreFactory<I, M, MessageStoreByPartition<I, M>>
+      CentralizedServiceWorker<I, V, E> service,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf,
+      MessageStoreFactory<I, Writable, MessageStoreByPartition<I, Writable>>
           messageStoreFactory,
       Mapper<?, ?, ?, ?>.Context context) {
-
+    this.conf = conf;
     this.messageStoreFactory = messageStoreFactory;
-    currentMessageStore = messageStoreFactory.newStore();
-    incomingMessageStore = messageStoreFactory.newStore();
+    currentMessageStore =
+        messageStoreFactory.newStore(conf.getOutgoingMessageValueClass());
+    incomingMessageStore =
+        messageStoreFactory.newStore(conf.getIncomingMessageValueClass());
     if (GiraphConstants.USE_OUT_OF_CORE_GRAPH.get(conf)) {
       partitionStore =
-          new DiskBackedPartitionStore<I, V, E, M>(conf, context);
+          new DiskBackedPartitionStore<I, V, E>(conf, context);
     } else {
       partitionStore =
-          new SimplePartitionStore<I, V, E, M>(conf, context);
+          new SimplePartitionStore<I, V, E>(conf, context);
     }
-    edgeStore = new EdgeStore<I, V, E, M>(service, conf, context);
+    edgeStore = new EdgeStore<I, V, E>(service, conf, context);
     ownerAggregatorData = new OwnerAggregatorServerData(context, conf);
     allAggregatorData = new AllAggregatorServerData(context, conf);
   }
 
-  public EdgeStore<I, V, E, M> getEdgeStore() {
+  public EdgeStore<I, V, E> getEdgeStore() {
     return edgeStore;
   }
 
@@ -119,7 +125,7 @@ public class ServerData<I extends WritableComparable,
    *
    * @return The partition store
    */
-  public PartitionStore<I, V, E, M> getPartitionStore() {
+  public PartitionStore<I, V, E> getPartitionStore() {
     return partitionStore;
   }
 
@@ -127,20 +133,24 @@ public class ServerData<I extends WritableComparable,
    * Get message store for incoming messages (messages which will be consumed
    * in the next super step)
    *
+   * @param <M> Message data
    * @return Incoming message store
    */
-  public MessageStoreByPartition<I, M> getIncomingMessageStore() {
-    return incomingMessageStore;
+  public <M extends Writable> MessageStoreByPartition<I, M>
+  getIncomingMessageStore() {
+    return (MessageStoreByPartition<I, M>) incomingMessageStore;
   }
 
   /**
    * Get message store for current messages (messages which we received in
    * previous super step and which will be consumed in current super step)
    *
+   * @param <M> Message data
    * @return Current message store
    */
-  public MessageStoreByPartition<I, M> getCurrentMessageStore() {
-    return currentMessageStore;
+  public <M extends Writable> MessageStoreByPartition<I, M>
+  getCurrentMessageStore() {
+    return (MessageStoreByPartition<I, M>) currentMessageStore;
   }
 
   /** Prepare for next super step */
@@ -154,7 +164,8 @@ public class ServerData<I extends WritableComparable,
       }
     }
     currentMessageStore = incomingMessageStore;
-    incomingMessageStore = messageStoreFactory.newStore();
+    incomingMessageStore =
+        messageStoreFactory.newStore(conf.getOutgoingMessageValueClass());
   }
 
   /**
@@ -162,7 +173,7 @@ public class ServerData<I extends WritableComparable,
    *
    * @return Vertex mutations
    */
-  public ConcurrentHashMap<I, VertexMutations<I, V, E, M>>
+  public ConcurrentHashMap<I, VertexMutations<I, V, E>>
   getVertexMutations() {
     return vertexMutations;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClient.java
index ebabf45..3759f6b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClient.java
@@ -33,11 +33,10 @@ import java.io.IOException;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface WorkerClient<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
 
   /**
    *  Setup the client.

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
index bc0637f..731d0ee 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
@@ -33,10 +33,9 @@ import java.io.IOException;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public interface WorkerClientRequestProcessor<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /**
    * Sends a message to destination vertex.
    *
@@ -44,7 +43,7 @@ public interface WorkerClientRequestProcessor<I extends WritableComparable,
    * @param message Message to send.
    * @return true if any network I/O occurred.
    */
-  boolean sendMessageRequest(I destVertexId, M message);
+  boolean sendMessageRequest(I destVertexId, Writable message);
 
   /**
    * Sends a vertex to the appropriate partition owner
@@ -53,7 +52,7 @@ public interface WorkerClientRequestProcessor<I extends WritableComparable,
    * @param vertex Vertex to send
    */
   void sendVertexRequest(PartitionOwner partitionOwner,
-                         Vertex<I, V, E, M> vertex);
+                         Vertex<I, V, E> vertex);
 
   /**
    * Send a partition request (no batching).
@@ -62,7 +61,7 @@ public interface WorkerClientRequestProcessor<I extends WritableComparable,
    * @param partition Partition to send
    */
   void sendPartitionRequest(WorkerInfo workerInfo,
-                            Partition<I, V, E, M> partition);
+                            Partition<I, V, E> partition);
 
   /**
    * Sends a request to the appropriate vertex range owner to add an edge
@@ -103,7 +102,7 @@ public interface WorkerClientRequestProcessor<I extends WritableComparable,
    * @param vertex Vertex to be added
    * @throws IOException
    */
-  void addVertexRequest(Vertex<I, V, E, M> vertex) throws IOException;
+  void addVertexRequest(Vertex<I, V, E> vertex) throws IOException;
 
   /**
    * Sends a request to the appropriate vertex range owner to remove a vertex

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/WorkerServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerServer.java
index e373b2c..bed07b9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerServer.java
@@ -18,7 +18,6 @@
 
 package org.apache.giraph.comm;
 
-import org.apache.giraph.graph.GraphState;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -31,11 +30,10 @@ import java.net.InetSocketAddress;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface WorkerServer<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     extends Closeable {
   /**
    * Get server address
@@ -46,17 +44,15 @@ public interface WorkerServer<I extends WritableComparable,
 
   /**
    * Prepare incoming messages for computation, and resolve mutation requests.
-   *
-   * @param graphState Current graph state
    */
-  void prepareSuperstep(GraphState<I, V, E, M> graphState);
+  void prepareSuperstep();
 
   /**
    * Get server data
    *
    * @return Server data
    */
-  ServerData<I, V, E, M> getServerData();
+  ServerData<I, V, E> getServerData();
 
   /**
    * Shuts down.


[07/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdEdges.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdEdges.java
index 2c5f2f7..762802b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdEdges.java
@@ -41,8 +41,8 @@ public class ByteArrayVertexIdEdges<I extends WritableComparable,
    * @return Casted configuration
    */
   @Override
-  public ImmutableClassesGiraphConfiguration<I, ?, E, ?> getConf() {
-    return (ImmutableClassesGiraphConfiguration<I, ?, E, ?>) super.getConf();
+  public ImmutableClassesGiraphConfiguration<I, ?, E> getConf() {
+    return (ImmutableClassesGiraphConfiguration<I, ?, E>) super.getConf();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdMessages.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdMessages.java b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdMessages.java
index 0280c58..6b4642c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdMessages.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ByteArrayVertexIdMessages.java
@@ -17,7 +17,6 @@
  */
 package org.apache.giraph.utils;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -34,10 +33,22 @@ import java.io.IOException;
 @SuppressWarnings("unchecked")
 public class ByteArrayVertexIdMessages<I extends WritableComparable,
     M extends Writable> extends ByteArrayVertexIdData<I, M> {
+  /** Message value class */
+  private Class<M> messageValueClass;
   /** Add the message size to the stream? (Depends on the message store) */
   private boolean useMessageSizeEncoding = false;
 
   /**
+   * Constructor
+   *
+   * @param messageValueClass Class for messages
+   */
+  public ByteArrayVertexIdMessages(
+      Class<? extends Writable> messageValueClass) {
+    this.messageValueClass = (Class<M>) messageValueClass;
+  }
+
+  /**
    * Set whether message sizes should be encoded.  This should only be a
    * possibility when not combining.  When combining, all messages need to be
    * deserializd right away, so this won't help.
@@ -50,20 +61,9 @@ public class ByteArrayVertexIdMessages<I extends WritableComparable,
     }
   }
 
-  /**
-   * Cast the {@link ImmutableClassesGiraphConfiguration} so it can be used
-   * to generate message objects.
-   *
-   * @return Casted configuration
-   */
-  @Override
-  public ImmutableClassesGiraphConfiguration<I, ?, ?, M> getConf() {
-    return (ImmutableClassesGiraphConfiguration<I, ?, ?, M>) super.getConf();
-  }
-
   @Override
   public M createData() {
-    return getConf().createMessageValue();
+    return ReflectionUtils.newInstance(messageValueClass);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
index 6016ba4..d8b121b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
@@ -28,6 +28,7 @@ import org.apache.commons.cli.ParseException;
 import org.apache.giraph.Algorithm;
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.edge.OutEdges;
@@ -142,14 +143,14 @@ public final class ConfigurationUtils {
     performSanityCheck(cmd);
 
     // Args are OK; attempt to populate the GiraphConfiguration with them.
-    final String vertexClassName = args[0];
+    final String computationClassName = args[0];
     final int workers = Integer.parseInt(cmd.getOptionValue('w'));
-    populateGiraphConfiguration(giraphConf, cmd, vertexClassName, workers);
+    populateGiraphConfiguration(giraphConf, cmd, computationClassName, workers);
 
     // validate generic parameters chosen are correct or
     // throw IllegalArgumentException, halting execution.
     @SuppressWarnings("rawtypes")
-    GiraphConfigurationValidator<?, ?, ?, ?> gtv =
+    GiraphConfigurationValidator<?, ?, ?, ?, ?> gtv =
       new GiraphConfigurationValidator(giraphConf);
     gtv.validateConfiguration();
 
@@ -200,15 +201,17 @@ public final class ConfigurationUtils {
    * should be captured here.
    * @param giraphConfiguration config for this job run
    * @param cmd parsed command line options to store in giraphConfiguration
-   * @param vertexClassName the vertex class (application) to run in this job.
+   * @param computationClassName the computation class (application) to run in
+   *                             this job.
    * @param workers the number of worker tasks for this job run.
    */
   private static void populateGiraphConfiguration(final GiraphConfiguration
-    giraphConfiguration, final CommandLine cmd, final String vertexClassName,
+    giraphConfiguration, final CommandLine cmd,
+      final String computationClassName,
     final int workers) throws ClassNotFoundException, IOException {
     giraphConfiguration.setWorkerConfiguration(workers, workers, 100.0f);
-    giraphConfiguration.setVertexClass(
-        (Class<? extends Vertex>) Class.forName(vertexClassName));
+    giraphConfiguration.setComputationClass(
+        (Class<? extends Computation>) Class.forName(computationClassName));
     if (cmd.hasOption("c")) {
       giraphConfiguration.setCombinerClass(
           (Class<? extends Combiner>) Class.forName(cmd.getOptionValue("c")));

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
index 65d99db..6ca488c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
@@ -76,9 +76,9 @@ public class InMemoryVertexInputFormat<I extends WritableComparable,
    */
   private class InMemoryVertexReader extends VertexReader<I, V, E> {
     /** The iterator */
-    private Iterator<Vertex<I, V, E, ?>> vertexIterator;
+    private Iterator<Vertex<I, V, E>> vertexIterator;
     /** Current vertex */
-    private Vertex<I, V, E, ?> currentVertex;
+    private Vertex<I, V, E> currentVertex;
 
     @Override
     public void initialize(InputSplit inputSplit,
@@ -96,7 +96,7 @@ public class InMemoryVertexInputFormat<I extends WritableComparable,
     }
 
     @Override
-    public Vertex<I, V, E, ?> getCurrentVertex() {
+    public Vertex<I, V, E> getCurrentVertex() {
       return currentVertex;
     }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
index be2d2a9..b4920e1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
@@ -101,7 +101,7 @@ public class InternalVertexRunner {
     File tmpDir = null;
     try {
       // Prepare input file, output folder and temporary folders
-      tmpDir = FileUtils.createTestDir(conf.getVertexClass());
+      tmpDir = FileUtils.createTestDir(conf.getComputationClass());
 
       File vertexInputFile = null;
       File edgeInputFile = null;
@@ -137,7 +137,7 @@ public class InternalVertexRunner {
       GiraphConstants.CHECKPOINT_DIRECTORY.set(conf, checkpointsDir.toString());
 
       // Create and configure the job to run the vertex
-      GiraphJob job = new GiraphJob(conf, conf.getVertexClass().getName());
+      GiraphJob job = new GiraphJob(conf, conf.getComputationClass().getName());
 
       Job internalJob = job.getInternalJob();
       if (conf.hasVertexInputFormat()) {
@@ -199,7 +199,6 @@ public class InternalVertexRunner {
    * @param <I> Vertex ID
    * @param <V> Vertex Value
    * @param <E> Edge Value
-   * @param <M> Message Value
    * @param conf GiraphClasses specifying which types to use
    * @param graph input graph
    * @return iterable output data
@@ -207,14 +206,13 @@ public class InternalVertexRunner {
    */
   public static <I extends WritableComparable,
     V extends Writable,
-    E extends Writable,
-    M extends Writable> TestGraph<I, V, E, M> run(
+    E extends Writable> TestGraph<I, V, E> run(
       GiraphConfiguration conf,
-      TestGraph<I, V, E, M> graph) throws Exception {
+      TestGraph<I, V, E> graph) throws Exception {
     File tmpDir = null;
     try {
       // Prepare temporary folders
-      tmpDir = FileUtils.createTestDir(conf.getVertexClass());
+      tmpDir = FileUtils.createTestDir(conf.getComputationClass());
 
       File zkDir = FileUtils.createTempDir(tmpDir, "_bspZooKeeper");
       File zkMgrDir = FileUtils.createTempDir(tmpDir, "_defaultZkManagerDir");
@@ -223,7 +221,7 @@ public class InternalVertexRunner {
       conf.setVertexInputFormatClass(InMemoryVertexInputFormat.class);
 
       // Create and configure the job to run the vertex
-      GiraphJob job = new GiraphJob(conf, conf.getVertexClass().getName());
+      GiraphJob job = new GiraphJob(conf, conf.getComputationClass().getName());
 
       InMemoryVertexInputFormat.setGraph(graph);
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ReflectionUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ReflectionUtils.java
index d70eecb..96352bb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ReflectionUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ReflectionUtils.java
@@ -168,6 +168,26 @@ public class ReflectionUtils {
   }
 
   /**
+   * Instantiate a class, wrap exceptions
+   *
+   * @param theClass Class to instantiate
+   * @param <T> Type to instantiate
+   * @return Newly instantiated object
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> T newInstance(Class<T> theClass) {
+    try {
+      return theClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new IllegalStateException(
+          "newInstance: Couldn't instantiate " + theClass.getName(), e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalStateException(
+          "newInstance: Illegal access " + theClass.getName(), e);
+    }
+  }
+
+  /**
    * Instantiate classes that are ImmutableClassesGiraphConfigurable
    *
    * @param theClass Class to instantiate

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java b/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
index 3577a9e..6e46a76 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
@@ -41,17 +41,15 @@ import java.util.Map.Entry;
  * @param <I> Vertex index type
  * @param <V> Vertex type
  * @param <E> Edge type
- * @param <M> Message type
  */
 public class TestGraph<I extends WritableComparable,
                        V extends Writable,
-                       E extends Writable,
-                       M extends Writable>
-                       implements Iterable<Vertex<I, V, E, M>> {
+                       E extends Writable>
+                       implements Iterable<Vertex<I, V, E>> {
   /** The vertex values */
-  private final HashMap<I, Vertex<I, V, E, M>> vertices = Maps.newHashMap();
+  private final HashMap<I, Vertex<I, V, E>> vertices = Maps.newHashMap();
   /** The configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
 
   /**
    * Constructor requiring classes
@@ -62,7 +60,7 @@ public class TestGraph<I extends WritableComparable,
     this.conf = new ImmutableClassesGiraphConfiguration(conf);
   }
 
-  public HashMap<I, Vertex<I, V, E, M>> getVertices() {
+  public HashMap<I, Vertex<I, V, E>> getVertices() {
     return vertices;
   }
 
@@ -82,9 +80,9 @@ public class TestGraph<I extends WritableComparable,
    * @param edges all edges
    * @return this
    */
-  public TestGraph<I, V, E, M> addVertex(I id, V value,
+  public TestGraph<I, V, E> addVertex(I id, V value,
                                          Entry<I, E>... edges) {
-    Vertex<I, V, E, M> v = makeVertex(id, value, edges);
+    Vertex<I, V, E> v = makeVertex(id, value, edges);
     vertices.put(id, v);
     return this;
   }
@@ -96,9 +94,9 @@ public class TestGraph<I extends WritableComparable,
    * @param edgePair The edge
    * @return this
    */
-  public TestGraph<I, V, E, M> addEdge(I vertexId, Entry<I, E> edgePair) {
+  public TestGraph<I, V, E> addEdge(I vertexId, Entry<I, E> edgePair) {
     if (!vertices.containsKey(vertexId)) {
-      Vertex<I, V, E, M> v = conf.createVertex();
+      Vertex<I, V, E> v = conf.createVertex();
       v.initialize(vertexId, conf.createVertexValue());
       vertices.put(vertexId, v);
     }
@@ -116,9 +114,9 @@ public class TestGraph<I extends WritableComparable,
    * @param edgeValue Edge value
    * @return this
    */
-  public TestGraph<I, V, E, M> addEdge(I vertexId, I toVertex, E edgeValue) {
+  public TestGraph<I, V, E> addEdge(I vertexId, I toVertex, E edgeValue) {
     if (!vertices.containsKey(vertexId)) {
-      Vertex<I, V, E, M> v = conf.createVertex();
+      Vertex<I, V, E> v = conf.createVertex();
       v.initialize(vertexId, conf.createVertexValue());
       vertices.put(vertexId, v);
     }
@@ -140,7 +138,7 @@ public class TestGraph<I extends WritableComparable,
    *
    * @return the iterator
    */
-  public Iterator<Vertex<I, V, E, M>> iterator() {
+  public Iterator<Vertex<I, V, E>> iterator() {
     return vertices.values().iterator();
   }
 
@@ -150,7 +148,7 @@ public class TestGraph<I extends WritableComparable,
    * @param id the id
    * @return the value
    */
-  public Vertex<I, V, E, M> getVertex(I id) {
+  public Vertex<I, V, E> getVertex(I id) {
     return vertices.get(id);
   }
 
@@ -177,10 +175,10 @@ public class TestGraph<I extends WritableComparable,
    * @param edges edges to other vertices
    * @return a new vertex
    */
-  protected Vertex<I, V, E, M> makeVertex(I id, V value,
+  protected Vertex<I, V, E> makeVertex(I id, V value,
       Entry<I, E>... edges) {
     @SuppressWarnings("unchecked")
-    Vertex<I, V, E, M> vertex = conf.createVertex();
+    Vertex<I, V, E> vertex = conf.createVertex();
     vertex.initialize(id, value, createEdges(edges));
     return vertex;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdIterator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdIterator.java b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdIterator.java
index 0c9ee07..bad11d6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdIterator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/VertexIdIterator.java
@@ -42,7 +42,7 @@ public abstract class VertexIdIterator<I extends WritableComparable> {
    */
   public VertexIdIterator(
       ExtendedDataOutput extendedDataOutput,
-      ImmutableClassesGiraphConfiguration<I, ?, ?, ?> configuration) {
+      ImmutableClassesGiraphConfiguration<I, ?, ?> configuration) {
     extendedDataInput = configuration.createExtendedDataInput(
         extendedDataOutput.getByteArray(), 0, extendedDataOutput.getPos());
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index c607ca3..c78d717 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -56,14 +56,16 @@ public class WritableUtils {
    * Read fields from byteArray to a Writeable object.
    *
    * @param byteArray Byte array to find the fields in.
-   * @param writableObject Object to fill in the fields.
+   * @param writableObjects Objects to fill in the fields.
    */
   public static void readFieldsFromByteArray(
-      byte[] byteArray, Writable writableObject) {
+      byte[] byteArray, Writable... writableObjects) {
     DataInputStream inputStream =
       new DataInputStream(new ByteArrayInputStream(byteArray));
     try {
-      writableObject.readFields(inputStream);
+      for (Writable writableObject : writableObjects) {
+        writableObject.readFields(inputStream);
+      }
     } catch (IOException e) {
       throw new IllegalStateException(
           "readFieldsFromByteArray: IOException", e);
@@ -77,16 +79,16 @@ public class WritableUtils {
    * @param zkPath Path of znode.
    * @param watch Add a watch?
    * @param stat Stat of znode if desired.
-   * @param writableObject Object to read into.
+   * @param writableObjects Objects to read into.
    */
   public static void readFieldsFromZnode(ZooKeeperExt zkExt,
                                          String zkPath,
                                          boolean watch,
                                          Stat stat,
-                                         Writable writableObject) {
+                                         Writable... writableObjects) {
     try {
       byte[] zkData = zkExt.getData(zkPath, false, stat);
-      readFieldsFromByteArray(zkData, writableObject);
+      readFieldsFromByteArray(zkData, writableObjects);
     } catch (KeeperException e) {
       throw new IllegalStateException(
         "readFieldsFromZnode: KeeperException on " + zkPath, e);
@@ -99,15 +101,17 @@ public class WritableUtils {
   /**
    * Write object to a byte array.
    *
-   * @param writableObject Object to write from.
+   * @param writableObjects Objects to write from.
    * @return Byte array with serialized object.
    */
-  public static byte[] writeToByteArray(Writable writableObject) {
+  public static byte[] writeToByteArray(Writable... writableObjects) {
     ByteArrayOutputStream outputStream =
         new ByteArrayOutputStream();
     DataOutput output = new DataOutputStream(outputStream);
     try {
-      writableObject.write(output);
+      for (Writable writableObject : writableObjects) {
+        writableObject.write(output);
+      }
     } catch (IOException e) {
       throw new IllegalStateException(
           "writeToByteArray: IOStateException", e);
@@ -189,15 +193,15 @@ public class WritableUtils {
    * @param zkExt ZooKeeper instance.
    * @param zkPath Path of znode.
    * @param version Version of the write.
-   * @param writableObject Object to write from.
+   * @param writableObjects Objects to write from.
    * @return Path and stat information of the znode.
    */
   public static PathStat writeToZnode(ZooKeeperExt zkExt,
                                       String zkPath,
                                       int version,
-                                      Writable writableObject) {
+                                      Writable... writableObjects) {
     try {
-      byte[] byteArray = writeToByteArray(writableObject);
+      byte[] byteArray = writeToByteArray(writableObjects);
       return zkExt.createOrSetExt(zkPath,
           byteArray,
           Ids.OPEN_ACL_UNSAFE,
@@ -341,15 +345,14 @@ public class WritableUtils {
    * @param <I> Vertex id
    * @param <V> Vertex value
    * @param <E> Edge value
-   * @param <M> Message value
    * @return Byte array with serialized object.
    */
   public static <I extends WritableComparable, V extends Writable,
-      E extends Writable, M extends Writable> byte[] writeVertexToByteArray(
-      Vertex<I, V, E, M> vertex,
+      E extends Writable> byte[] writeVertexToByteArray(
+      Vertex<I, V, E> vertex,
       byte[] buffer,
       boolean unsafe,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     ExtendedDataOutput extendedDataOutput;
     if (unsafe) {
       extendedDataOutput = new UnsafeByteArrayOutputStream(buffer);
@@ -378,14 +381,13 @@ public class WritableUtils {
    * @param <I> Vertex id
    * @param <V> Vertex value
    * @param <E> Edge value
-   * @param <M> Message value
    * @return Byte array with serialized object.
    */
   public static <I extends WritableComparable, V extends Writable,
-      E extends Writable, M extends Writable> byte[] writeVertexToByteArray(
-      Vertex<I, V, E, M> vertex,
+      E extends Writable> byte[] writeVertexToByteArray(
+      Vertex<I, V, E> vertex,
       boolean unsafe,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     return writeVertexToByteArray(vertex, null, unsafe, conf);
   }
 
@@ -400,16 +402,14 @@ public class WritableUtils {
   * @param <I> Vertex id
   * @param <V> Vertex value
   * @param <E> Edge value
-  * @param <M> Message value
   * @param conf Configuration
   */
   public static <I extends WritableComparable, V extends Writable,
-  E extends Writable, M extends Writable> void
-  reinitializeVertexFromByteArray(
+  E extends Writable> void reinitializeVertexFromByteArray(
       byte[] byteArray,
-      Vertex<I, V, E, M> vertex,
+      Vertex<I, V, E> vertex,
       boolean unsafe,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     ExtendedDataInput extendedDataInput;
     if (unsafe) {
       extendedDataInput = new UnsafeByteArrayInputStream(byteArray);
@@ -465,15 +465,14 @@ public class WritableUtils {
    * @param <I> Vertex id
    * @param <V> Vertex value
    * @param <E> Edge value
-   * @param <M> Message value
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
   public static <I extends WritableComparable, V extends Writable,
-  E extends Writable, M extends Writable> void reinitializeVertexFromDataInput(
+  E extends Writable> void reinitializeVertexFromDataInput(
       DataInput input,
-      Vertex<I, V, E, M> vertex,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf)
+      Vertex<I, V, E> vertex,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf)
     throws IOException {
     vertex.getId().readFields(input);
     vertex.getValue().readFields(input);
@@ -493,17 +492,16 @@ public class WritableUtils {
    * @param <I> Vertex id
    * @param <V> Vertex value
    * @param <E> Edge value
-   * @param <M> Message value
    * @return The vertex
    * @throws IOException
    */
   public static <I extends WritableComparable, V extends Writable,
-  E extends Writable, M extends Writable> Vertex<I, V, E, M>
+  E extends Writable> Vertex<I, V, E>
   readVertexFromDataInput(
       DataInput input,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf)
+      ImmutableClassesGiraphConfiguration<I, V, E> conf)
     throws IOException {
-    Vertex<I, V, E, M> vertex = conf.createVertex();
+    Vertex<I, V, E> vertex = conf.createVertex();
     I id = conf.createVertexId();
     V value = conf.createVertexValue();
     OutEdges<I, E> edges = conf.createOutEdges();
@@ -521,19 +519,56 @@ public class WritableUtils {
    * @param <I> Vertex id
    * @param <V> Vertex value
    * @param <E> Edge value
-   * @param <M> Message value
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
   public static <I extends WritableComparable, V extends Writable,
-  E extends Writable, M extends Writable> void writeVertexToDataOutput(
+  E extends Writable> void writeVertexToDataOutput(
       DataOutput output,
-      Vertex<I, V, E, M> vertex,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf)
+      Vertex<I, V, E> vertex,
+      ImmutableClassesGiraphConfiguration<I, V, E> conf)
     throws IOException {
     vertex.getId().write(output);
     vertex.getValue().write(output);
     ((OutEdges<I, E>) vertex.getEdges()).write(output);
     output.writeBoolean(vertex.isHalted());
   }
+
+  /**
+   * Write class to data output. Also handles the case when class is null.
+   *
+   * @param clazz Class
+   * @param output Data output
+   * @param <T> Class type
+   */
+  public static <T> void writeClass(Class<T> clazz,
+      DataOutput output) throws IOException {
+    output.writeBoolean(clazz != null);
+    if (clazz != null) {
+      output.writeUTF(clazz.getName());
+    }
+  }
+
+  /**
+   * Read class from data input.
+   * Matches {@link #writeClass(Class, DataOutput)}.
+   *
+   * @param input Data input
+   * @param <T> Class type
+   * @return Class, or null if null was written
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> Class<T> readClass(DataInput input) throws IOException {
+    if (input.readBoolean()) {
+      String className = input.readUTF();
+      try {
+        return (Class<T>) Class.forName(className);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalStateException("readClass: No class found " +
+            className);
+      }
+    } else {
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
index 03a4876..8b5e39a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java
@@ -35,7 +35,6 @@ import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.AddressesAndPartitionsWritable;
 import org.apache.giraph.graph.FinishedSuperstepStats;
 import org.apache.giraph.graph.GlobalStats;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.GraphTaskManager;
 import org.apache.giraph.graph.InputSplitEvents;
 import org.apache.giraph.graph.InputSplitPaths;
@@ -45,6 +44,7 @@ import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.giraph.io.VertexWriter;
 import org.apache.giraph.io.superstep_output.SuperstepOutput;
 import org.apache.giraph.master.MasterInfo;
+import org.apache.giraph.master.SuperstepClasses;
 import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.giraph.metrics.GiraphTimer;
 import org.apache.giraph.metrics.GiraphTimerContext;
@@ -108,13 +108,12 @@ import java.util.concurrent.TimeUnit;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class BspServiceWorker<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends BspService<I, V, E, M>
-    implements CentralizedServiceWorker<I, V, E, M>,
+    V extends Writable, E extends Writable>
+    extends BspService<I, V, E>
+    implements CentralizedServiceWorker<I, V, E>,
     ResetSuperstepMetricsObserver {
   /** Name of gauge for time spent waiting on other workers */
   public static final String TIMER_WAIT_REQUESTS = "wait-requests-us";
@@ -125,12 +124,12 @@ public class BspServiceWorker<I extends WritableComparable,
   /** Worker info */
   private final WorkerInfo workerInfo;
   /** Worker graph partitioner */
-  private final WorkerGraphPartitioner<I, V, E, M> workerGraphPartitioner;
+  private final WorkerGraphPartitioner<I, V, E> workerGraphPartitioner;
 
   /** IPC Client */
-  private final WorkerClient<I, V, E, M> workerClient;
+  private final WorkerClient<I, V, E> workerClient;
   /** IPC Server */
-  private final WorkerServer<I, V, E, M> workerServer;
+  private final WorkerServer<I, V, E> workerServer;
   /** Request processor for aggregator requests */
   private final WorkerAggregatorRequestProcessor
   workerAggregatorRequestProcessor;
@@ -173,27 +172,28 @@ public class BspServiceWorker<I extends WritableComparable,
     String serverPortList,
     int sessionMsecTimeout,
     Mapper<?, ?, ?, ?>.Context context,
-    GraphTaskManager<I, V, E, M> graphTaskManager)
+    GraphTaskManager<I, V, E> graphTaskManager)
     throws IOException, InterruptedException {
     super(serverPortList, sessionMsecTimeout, context, graphTaskManager);
-    ImmutableClassesGiraphConfiguration<I, V, E, M> conf = getConfiguration();
+    ImmutableClassesGiraphConfiguration<I, V, E> conf = getConfiguration();
     partitionExchangeChildrenChanged = new PredicateLock(context);
     registerBspEvent(partitionExchangeChildrenChanged);
     workerGraphPartitioner =
         getGraphPartitionerFactory().createWorkerGraphPartitioner();
     workerInfo = new WorkerInfo();
-    workerServer = new NettyWorkerServer<I, V, E, M>(conf, this, context);
+    workerServer = new NettyWorkerServer<I, V, E>(conf, this, context);
     workerInfo.setInetSocketAddress(workerServer.getMyAddress());
     workerInfo.setTaskId(getTaskPartition());
-    workerClient = new NettyWorkerClient<I, V, E, M>(context, conf, this);
+    workerClient = new NettyWorkerClient<I, V, E>(context, conf, this);
 
     workerAggregatorRequestProcessor =
         new NettyWorkerAggregatorRequestProcessor(getContext(), conf, this);
 
-    workerContext = conf.createWorkerContext(null);
-
     aggregatorHandler = new WorkerAggregatorHandler(this, conf, context);
 
+    workerContext = conf.createWorkerContext();
+    workerContext.setWorkerAggregatorUsage(aggregatorHandler);
+
     superstepOutput = conf.createSuperstepOutput(context);
 
     if (conf.isJMapHistogramDumpEnabled()) {
@@ -223,7 +223,7 @@ public class BspServiceWorker<I extends WritableComparable,
   }
 
   @Override
-  public WorkerClient<I, V, E, M> getWorkerClient() {
+  public WorkerClient<I, V, E> getWorkerClient() {
     return workerClient;
   }
 
@@ -285,7 +285,8 @@ public class BspServiceWorker<I extends WritableComparable,
 
 
   /**
-   * Load the vertices from the user-defined {@link VertexReader}
+   * Load the vertices from the user-defined
+   * {@link org.apache.giraph.io.VertexReader}
    *
    * @return Count of vertices and edges loaded
    */
@@ -295,10 +296,6 @@ public class BspServiceWorker<I extends WritableComparable,
         getZkExt().getChildrenExt(vertexInputSplitsPaths.getPath(),
             false, false, true);
 
-    GraphState<I, V, E, M> graphState = new GraphState<I, V, E, M>(
-        INPUT_SUPERSTEP, 0, 0, getContext(), getGraphTaskManager(),
-        null, null);
-
     InputSplitPathOrganizer splitOrganizer =
         new InputSplitPathOrganizer(getZkExt(),
             inputSplitPathList, getWorkerInfo().getHostname(),
@@ -310,11 +307,10 @@ public class BspServiceWorker<I extends WritableComparable,
         BspService.VERTEX_INPUT_SPLIT_RESERVED_NODE,
         BspService.VERTEX_INPUT_SPLIT_FINISHED_NODE);
 
-    VertexInputSplitsCallableFactory<I, V, E, M> inputSplitsCallableFactory =
-        new VertexInputSplitsCallableFactory<I, V, E, M>(
+    VertexInputSplitsCallableFactory<I, V, E> inputSplitsCallableFactory =
+        new VertexInputSplitsCallableFactory<I, V, E>(
             getConfiguration().createWrappedVertexInputFormat(),
             getContext(),
-            graphState,
             getConfiguration(),
             this,
             splitsHandler,
@@ -324,7 +320,8 @@ public class BspServiceWorker<I extends WritableComparable,
   }
 
   /**
-   * Load the edges from the user-defined {@link EdgeReader}.
+   * Load the edges from the user-defined
+   * {@link org.apache.giraph.io.EdgeReader}.
    *
    * @return Number of edges loaded
    */
@@ -333,10 +330,6 @@ public class BspServiceWorker<I extends WritableComparable,
         getZkExt().getChildrenExt(edgeInputSplitsPaths.getPath(),
             false, false, true);
 
-    GraphState<I, V, E, M> graphState = new GraphState<I, V, E, M>(
-        INPUT_SUPERSTEP, 0, 0, getContext(), getGraphTaskManager(),
-        null, null);
-
     InputSplitPathOrganizer splitOrganizer =
         new InputSplitPathOrganizer(getZkExt(),
             inputSplitPathList, getWorkerInfo().getHostname(),
@@ -348,11 +341,10 @@ public class BspServiceWorker<I extends WritableComparable,
         BspService.EDGE_INPUT_SPLIT_RESERVED_NODE,
         BspService.EDGE_INPUT_SPLIT_FINISHED_NODE);
 
-    EdgeInputSplitsCallableFactory<I, V, E, M> inputSplitsCallableFactory =
-        new EdgeInputSplitsCallableFactory<I, V, E, M>(
+    EdgeInputSplitsCallableFactory<I, V, E> inputSplitsCallableFactory =
+        new EdgeInputSplitsCallableFactory<I, V, E>(
             getConfiguration().createWrappedEdgeInputFormat(),
             getContext(),
-            graphState,
             getConfiguration(),
             this,
             splitsHandler,
@@ -483,11 +475,8 @@ public class BspServiceWorker<I extends WritableComparable,
     }
 
     // Add the partitions that this worker owns
-    GraphState<I, V, E, M> graphState =
-        new GraphState<I, V, E, M>(INPUT_SUPERSTEP, 0, 0,
-            getContext(), getGraphTaskManager(), null, null);
     Collection<? extends PartitionOwner> masterSetPartitionOwners =
-        startSuperstep(graphState);
+        startSuperstep();
     workerGraphPartitioner.updatePartitionOwners(
         getWorkerInfo(), masterSetPartitionOwners, getPartitionStore());
 
@@ -552,7 +541,7 @@ else[HADOOP_NON_SECURE]*/
       if (partitionOwner.getWorkerInfo().equals(getWorkerInfo()) &&
           !getPartitionStore().hasPartition(
               partitionOwner.getPartitionId())) {
-        Partition<I, V, E, M> partition =
+        Partition<I, V, E> partition =
             getConfiguration().createPartition(
                 partitionOwner.getPartitionId(), getContext());
         getPartitionStore().addPartition(partition);
@@ -569,7 +558,7 @@ else[HADOOP_NON_SECURE]*/
     List<PartitionStats> partitionStatsList =
         new ArrayList<PartitionStats>();
     for (Integer partitionId : getPartitionStore().getPartitionIds()) {
-      Partition<I, V, E, M> partition =
+      Partition<I, V, E> partition =
           getPartitionStore().getPartition(partitionId);
       PartitionStats partitionStats =
           new PartitionStats(partition.getId(),
@@ -583,7 +572,7 @@ else[HADOOP_NON_SECURE]*/
     workerGraphPartitioner.finalizePartitionStats(
         partitionStatsList, getPartitionStore());
 
-    return finishSuperstep(graphState, partitionStatsList);
+    return finishSuperstep(partitionStatsList);
   }
 
   /**
@@ -666,8 +655,7 @@ else[HADOOP_NON_SECURE]*/
   }
 
   @Override
-  public Collection<? extends PartitionOwner> startSuperstep(
-      GraphState<I, V, E, M> graphState) {
+  public Collection<? extends PartitionOwner> startSuperstep() {
     // Algorithm:
     // 1. Communication service will combine message from previous
     //    superstep
@@ -675,7 +663,7 @@ else[HADOOP_NON_SECURE]*/
     // 3. Wait until the partition assignment is complete and get it
     // 4. Get the aggregator values from the previous superstep
     if (getSuperstep() != INPUT_SUPERSTEP) {
-      workerServer.prepareSuperstep(graphState);
+      workerServer.prepareSuperstep();
     }
 
     registerHealth(getSuperstep());
@@ -727,7 +715,6 @@ else[HADOOP_NON_SECURE]*/
 
   @Override
   public FinishedSuperstepStats finishSuperstep(
-      GraphState<I, V, E, M> graphState,
       List<PartitionStats> partitionStatsList) {
     // This barrier blocks until success (or the master signals it to
     // restart).
@@ -740,10 +727,10 @@ else[HADOOP_NON_SECURE]*/
     // 4. Report the statistics (vertices, edges, messages, etc.)
     //    of this worker
     // 5. Let the master know it is finished.
-    // 6. Wait for the master's global stats, and check if done
+    // 6. Wait for the master's superstep info, and check if done
     waitForRequestsToFinish();
 
-    graphState.getGraphTaskManager().notifyFinishedCommunication();
+    getGraphTaskManager().notifyFinishedCommunication();
 
     long workerSentMessages = 0;
     long localVertices = 0;
@@ -753,7 +740,7 @@ else[HADOOP_NON_SECURE]*/
     }
 
     if (getSuperstep() != INPUT_SUPERSTEP) {
-      postSuperstepCallbacks(graphState);
+      postSuperstepCallbacks();
     }
 
     aggregatorHandler.finishSuperstep(workerAggregatorRequestProcessor);
@@ -779,17 +766,21 @@ else[HADOOP_NON_SECURE]*/
     waitForOtherWorkers(superstepFinishedNode);
 
     GlobalStats globalStats = new GlobalStats();
+    SuperstepClasses superstepClasses = new SuperstepClasses();
     WritableUtils.readFieldsFromZnode(
-        getZkExt(), superstepFinishedNode, false, null, globalStats);
+        getZkExt(), superstepFinishedNode, false, null, globalStats,
+        superstepClasses);
     if (LOG.isInfoEnabled()) {
       LOG.info("finishSuperstep: Completed superstep " + getSuperstep() +
-          " with global stats " + globalStats);
+          " with global stats " + globalStats + " and classes " +
+          superstepClasses);
     }
     incrCachedSuperstep();
     getContext().setStatus("finishSuperstep: (all workers done) " +
         getGraphTaskManager().getGraphFunctions().toString() +
         " - Attempt=" + getApplicationAttempt() +
         ", Superstep=" + getSuperstep());
+    getConfiguration().updateSuperstepClasses(superstepClasses);
 
     return new FinishedSuperstepStats(
         localVertices,
@@ -801,18 +792,15 @@ else[HADOOP_NON_SECURE]*/
 
   /**
    * Handle post-superstep callbacks
-   *
-   * @param graphState GraphState
    */
-  private void postSuperstepCallbacks(GraphState<I, V, E, M> graphState) {
-    getWorkerContext().setGraphState(graphState);
+  private void postSuperstepCallbacks() {
     GiraphTimerContext timerContext = wcPostSuperstepTimer.time();
     getWorkerContext().postSuperstep();
     timerContext.stop();
     getContext().progress();
 
     for (WorkerObserver obs : getWorkerObservers()) {
-      obs.postSuperstep(graphState.getSuperstep());
+      obs.postSuperstep(getSuperstep());
       getContext().progress();
     }
   }
@@ -943,9 +931,7 @@ else[HADOOP_NON_SECURE]*/
           public Void call() throws Exception {
             VertexWriter<I, V, E> vertexWriter =
                 vertexOutputFormat.createVertexWriter(getContext());
-            vertexWriter.setConf(
-                (ImmutableClassesGiraphConfiguration<I, V, E, Writable>)
-                    getConfiguration());
+            vertexWriter.setConf(getConfiguration());
             vertexWriter.initialize(getContext());
             long verticesWritten = 0;
             long nextPrintVertices = 0;
@@ -953,9 +939,9 @@ else[HADOOP_NON_SECURE]*/
             int partitionIndex = 0;
             int numPartitions = getPartitionStore().getNumPartitions();
             for (Integer partitionId : getPartitionStore().getPartitionIds()) {
-              Partition<I, V, E, M> partition =
+              Partition<I, V, E> partition =
                   getPartitionStore().getPartition(partitionId);
-              for (Vertex<I, V, E, M> vertex : partition) {
+              for (Vertex<I, V, E> vertex : partition) {
                 vertexWriter.writeVertex(vertex);
                 ++verticesWritten;
 
@@ -1104,7 +1090,7 @@ else[HADOOP_NON_SECURE]*/
     ByteArrayOutputStream metadataByteStream = new ByteArrayOutputStream();
     DataOutput metadataOutput = new DataOutputStream(metadataByteStream);
     for (Integer partitionId : getPartitionStore().getPartitionIds()) {
-      Partition<I, V, E, M> partition =
+      Partition<I, V, E> partition =
           getPartitionStore().getPartition(partitionId);
       long startPos = verticesOutputStream.getPos();
       partition.write(verticesOutputStream);
@@ -1209,7 +1195,7 @@ else[HADOOP_NON_SECURE]*/
                 " not found!");
           }
           metadataStream.close();
-          Partition<I, V, E, M> partition =
+          Partition<I, V, E> partition =
               getConfiguration().createPartition(partitionId, getContext());
           DataInputStream partitionsStream =
               getFs().open(new Path(partitionsFile));
@@ -1250,18 +1236,21 @@ else[HADOOP_NON_SECURE]*/
           " total.");
     }
 
-    // Load global statistics
-    GlobalStats globalStats = null;
+    // Load global stats and superstep classes
+    GlobalStats globalStats = new GlobalStats();
+    SuperstepClasses superstepClasses = new SuperstepClasses();
     String finalizedCheckpointPath =
         getCheckpointBasePath(superstep) + CHECKPOINT_FINALIZED_POSTFIX;
     try {
       DataInputStream finalizedStream =
           getFs().open(new Path(finalizedCheckpointPath));
-      globalStats = new GlobalStats();
       globalStats.readFields(finalizedStream);
+      superstepClasses.readFields(finalizedStream);
+      getConfiguration().updateSuperstepClasses(superstepClasses);
     } catch (IOException e) {
       throw new IllegalStateException(
-          "loadCheckpoint: Failed to load global statistics", e);
+          "loadCheckpoint: Failed to load global stats and superstep classes",
+          e);
     }
 
     // Communication service needs to setup the connections prior to
@@ -1287,13 +1276,13 @@ else[HADOOP_NON_SECURE]*/
         new ArrayList<Entry<WorkerInfo, List<Integer>>>(
             workerPartitionMap.entrySet());
     Collections.shuffle(randomEntryList);
-    WorkerClientRequestProcessor<I, V, E, M> workerClientRequestProcessor =
-        new NettyWorkerClientRequestProcessor<I, V, E, M>(getContext(),
+    WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor =
+        new NettyWorkerClientRequestProcessor<I, V, E>(getContext(),
             getConfiguration(), this);
     for (Entry<WorkerInfo, List<Integer>> workerPartitionList :
       randomEntryList) {
       for (Integer partitionId : workerPartitionList.getValue()) {
-        Partition<I, V, E, M> partition =
+        Partition<I, V, E> partition =
             getPartitionStore().removePartition(partitionId);
         if (partition == null) {
           throw new IllegalStateException(
@@ -1467,7 +1456,7 @@ else[HADOOP_NON_SECURE]*/
   }
 
   @Override
-  public PartitionStore<I, V, E, M> getPartitionStore() {
+  public PartitionStore<I, V, E> getPartitionStore() {
     return getServerData().getPartitionStore();
   }
 
@@ -1493,7 +1482,7 @@ else[HADOOP_NON_SECURE]*/
   }
 
   @Override
-  public ServerData<I, V, E, M> getServerData() {
+  public ServerData<I, V, E> getServerData() {
     return workerServer.getServerData();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java b/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
index 351a114..78cdd8e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallable.java
@@ -20,7 +20,6 @@ package org.apache.giraph.worker;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.VertexEdgeCount;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.EdgeReader;
@@ -48,11 +47,10 @@ import java.io.IOException;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class EdgeInputSplitsCallable<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends InputSplitsCallable<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends InputSplitsCallable<I, V, E> {
   /** How often to update metrics and print info */
   public static final int EDGES_UPDATE_PERIOD = 1000000;
   /** How often to update filtered metrics */
@@ -80,7 +78,6 @@ public class EdgeInputSplitsCallable<I extends WritableComparable,
    *
    * @param edgeInputFormat Edge input format
    * @param context Context
-   * @param graphState Graph state
    * @param configuration Configuration
    * @param bspServiceWorker service worker
    * @param splitsHandler Handler for input splits
@@ -89,13 +86,12 @@ public class EdgeInputSplitsCallable<I extends WritableComparable,
   public EdgeInputSplitsCallable(
       EdgeInputFormat<I, E> edgeInputFormat,
       Mapper<?, ?, ?, ?>.Context context,
-      GraphState<I, V, E, M> graphState,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
-      BspServiceWorker<I, V, E, M> bspServiceWorker,
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
+      BspServiceWorker<I, V, E> bspServiceWorker,
       InputSplitsHandler splitsHandler,
       ZooKeeperExt zooKeeperExt)  {
-    super(context, graphState, configuration, bspServiceWorker,
-        splitsHandler, zooKeeperExt);
+    super(context, configuration, bspServiceWorker, splitsHandler,
+        zooKeeperExt);
     this.edgeInputFormat = edgeInputFormat;
 
     inputSplitMaxEdges = configuration.getInputSplitMaxEdges();
@@ -116,20 +112,18 @@ public class EdgeInputSplitsCallable<I extends WritableComparable,
    * maximum number of edges to be read from an input split.
    *
    * @param inputSplit Input split to process with edge reader
-   * @param graphState Current graph state
    * @return Edges loaded from this input split
    * @throws IOException
    * @throws InterruptedException
    */
   @Override
   protected VertexEdgeCount readInputSplit(
-      InputSplit inputSplit,
-      GraphState<I, V, E, M> graphState) throws IOException,
+      InputSplit inputSplit) throws IOException,
       InterruptedException {
     EdgeReader<I, E> edgeReader =
         edgeInputFormat.createEdgeReader(inputSplit, context);
     edgeReader.setConf(
-        (ImmutableClassesGiraphConfiguration<I, Writable, E, Writable>)
+        (ImmutableClassesGiraphConfiguration<I, Writable, E>)
             configuration);
     edgeReader.initialize(inputSplit, context);
 
@@ -166,8 +160,7 @@ public class EdgeInputSplitsCallable<I extends WritableComparable,
         continue;
       }
 
-      graphState.getWorkerClientRequestProcessor().sendEdgeRequest(sourceId,
-          readerEdge);
+      workerClientRequestProcessor.sendEdgeRequest(sourceId, readerEdge);
       context.progress(); // do this before potential data transfer
 
       // Update status every EDGES_UPDATE_PERIOD edges

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallableFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallableFactory.java b/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallableFactory.java
index 33fb515..f68ac93 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallableFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/EdgeInputSplitsCallableFactory.java
@@ -19,7 +19,6 @@
 package org.apache.giraph.worker;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.VertexEdgeCount;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.utils.CallableFactory;
@@ -34,21 +33,18 @@ import org.apache.hadoop.mapreduce.Mapper;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class EdgeInputSplitsCallableFactory<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     implements CallableFactory<VertexEdgeCount> {
   /** Edge input format */
   private final EdgeInputFormat<I, E> edgeInputFormat;
   /** Mapper context. */
   private final Mapper<?, ?, ?, ?>.Context context;
-  /** Graph state. */
-  private final GraphState<I, V, E, M> graphState;
   /** Configuration. */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /** {@link BspServiceWorker} we're running on. */
-  private final BspServiceWorker<I, V, E, M> bspServiceWorker;
+  private final BspServiceWorker<I, V, E> bspServiceWorker;
   /** Handler for input splits */
   private final InputSplitsHandler splitsHandler;
   /** {@link ZooKeeperExt} for this worker. */
@@ -59,7 +55,6 @@ public class EdgeInputSplitsCallableFactory<I extends WritableComparable,
    *
    * @param edgeInputFormat Edge input format
    * @param context Mapper context
-   * @param graphState Graph state
    * @param configuration Configuration
    * @param bspServiceWorker Calling {@link BspServiceWorker}
    * @param splitsHandler Handler for input splits
@@ -68,14 +63,12 @@ public class EdgeInputSplitsCallableFactory<I extends WritableComparable,
   public EdgeInputSplitsCallableFactory(
       EdgeInputFormat<I, E> edgeInputFormat,
       Mapper<?, ?, ?, ?>.Context context,
-      GraphState<I, V, E, M> graphState,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
-      BspServiceWorker<I, V, E, M> bspServiceWorker,
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
+      BspServiceWorker<I, V, E> bspServiceWorker,
       InputSplitsHandler splitsHandler,
       ZooKeeperExt zooKeeperExt) {
     this.edgeInputFormat = edgeInputFormat;
     this.context = context;
-    this.graphState = graphState;
     this.configuration = configuration;
     this.bspServiceWorker = bspServiceWorker;
     this.zooKeeperExt = zooKeeperExt;
@@ -83,11 +76,10 @@ public class EdgeInputSplitsCallableFactory<I extends WritableComparable,
   }
 
   @Override
-  public InputSplitsCallable<I, V, E, M> newCallable(int threadId) {
-    return new EdgeInputSplitsCallable<I, V, E, M>(
+  public InputSplitsCallable<I, V, E> newCallable(int threadId) {
+    return new EdgeInputSplitsCallable<I, V, E>(
         edgeInputFormat,
         context,
-        graphState,
         configuration,
         bspServiceWorker,
         splitsHandler,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/InputSplitsCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/InputSplitsCallable.java b/giraph-core/src/main/java/org/apache/giraph/worker/InputSplitsCallable.java
index a8298c5..10b1a25 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/InputSplitsCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/InputSplitsCallable.java
@@ -21,7 +21,6 @@ package org.apache.giraph.worker;
 import org.apache.giraph.comm.WorkerClientRequestProcessor;
 import org.apache.giraph.comm.netty.NettyWorkerClientRequestProcessor;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.VertexEdgeCount;
 import org.apache.giraph.io.GiraphInputFormat;
 import org.apache.giraph.metrics.GiraphMetrics;
@@ -57,24 +56,20 @@ import java.util.concurrent.Callable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public abstract class InputSplitsCallable<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     implements Callable<VertexEdgeCount> {
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(InputSplitsCallable.class);
   /** Class time object */
   private static final Time TIME = SystemTime.get();
   /** Configuration */
-  protected final ImmutableClassesGiraphConfiguration<I, V, E, M>
-  configuration;
+  protected final ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /** Context */
   protected final Mapper<?, ?, ?, ?>.Context context;
-  /** Graph state */
-  private final GraphState<I, V, E, M> graphState;
   /** Handles IPC communication */
-  private final WorkerClientRequestProcessor<I, V, E, M>
+  protected final WorkerClientRequestProcessor<I, V, E>
   workerClientRequestProcessor;
   /**
    * Stores and processes the list of InputSplits advertised
@@ -93,7 +88,6 @@ public abstract class InputSplitsCallable<I extends WritableComparable,
    * Constructor.
    *
    * @param context Context
-   * @param graphState Graph state
    * @param configuration Configuration
    * @param bspServiceWorker service worker
    * @param splitsHandler Handler for input splits
@@ -101,20 +95,15 @@ public abstract class InputSplitsCallable<I extends WritableComparable,
    */
   public InputSplitsCallable(
       Mapper<?, ?, ?, ?>.Context context,
-      GraphState<I, V, E, M> graphState,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
-      BspServiceWorker<I, V, E, M> bspServiceWorker,
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
+      BspServiceWorker<I, V, E> bspServiceWorker,
       InputSplitsHandler splitsHandler,
       ZooKeeperExt zooKeeperExt) {
     this.zooKeeperExt = zooKeeperExt;
     this.context = context;
     this.workerClientRequestProcessor =
-        new NettyWorkerClientRequestProcessor<I, V, E, M>(
+        new NettyWorkerClientRequestProcessor<I, V, E>(
             context, configuration, bspServiceWorker);
-    this.graphState = new GraphState<I, V, E, M>(graphState.getSuperstep(),
-        graphState.getTotalNumVertices(), graphState.getTotalNumEdges(),
-        context, graphState.getGraphTaskManager(), workerClientRequestProcessor,
-        null);
     this.useLocality = configuration.useInputSplitLocality();
     this.splitsHandler = splitsHandler;
     this.configuration = configuration;
@@ -205,14 +194,11 @@ public abstract class InputSplitsCallable<I extends WritableComparable,
    * Load vertices/edges from the given input split.
    *
    * @param inputSplit Input split to load
-   * @param graphState Graph state
    * @return Count of vertices and edges loaded
    * @throws IOException
    * @throws InterruptedException
    */
-  protected abstract VertexEdgeCount readInputSplit(
-      InputSplit inputSplit,
-      GraphState<I, V, E, M> graphState)
+  protected abstract VertexEdgeCount readInputSplit(InputSplit inputSplit)
     throws IOException, InterruptedException;
 
   @Override
@@ -222,9 +208,8 @@ public abstract class InputSplitsCallable<I extends WritableComparable,
     int inputSplitsProcessed = 0;
     try {
       while ((inputSplitPath = splitsHandler.reserveInputSplit()) != null) {
-        vertexEdgeCount = vertexEdgeCount.incrVertexEdgeCount(
-            loadInputSplit(inputSplitPath,
-                graphState));
+        vertexEdgeCount =
+            vertexEdgeCount.incrVertexEdgeCount(loadInputSplit(inputSplitPath));
         context.progress();
         ++inputSplitsProcessed;
       }
@@ -267,7 +252,6 @@ public abstract class InputSplitsCallable<I extends WritableComparable,
    * Mark the input split finished when done.
    *
    * @param inputSplitPath ZK location of input split
-   * @param graphState Current graph state
    * @return Mapping of vertex indices and statistics, or null if no data read
    * @throws IOException
    * @throws ClassNotFoundException
@@ -276,13 +260,11 @@ public abstract class InputSplitsCallable<I extends WritableComparable,
    * @throws IllegalAccessException
    */
   private VertexEdgeCount loadInputSplit(
-      String inputSplitPath,
-      GraphState<I, V, E, M> graphState)
+      String inputSplitPath)
     throws IOException, ClassNotFoundException, InterruptedException,
       InstantiationException, IllegalAccessException {
     InputSplit inputSplit = getInputSplit(inputSplitPath);
-    VertexEdgeCount vertexEdgeCount =
-        readInputSplit(inputSplit, graphState);
+    VertexEdgeCount vertexEdgeCount = readInputSplit(inputSplit);
     if (LOG.isInfoEnabled()) {
       LOG.info("loadFromInputSplit: Finished loading " +
           inputSplitPath + " " + vertexEdgeCount);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java b/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
index 1c292ad..977e100 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallable.java
@@ -19,7 +19,6 @@
 package org.apache.giraph.worker;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexEdgeCount;
 import org.apache.giraph.io.GiraphInputFormat;
@@ -50,11 +49,10 @@ import java.io.IOException;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class VertexInputSplitsCallable<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends InputSplitsCallable<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends InputSplitsCallable<I, V, E> {
   /** How often to update metrics and print info */
   public static final int VERTICES_UPDATE_PERIOD = 250000;
   /** How often to update filtered out metrics */
@@ -68,9 +66,9 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
   /** Input split max vertices (-1 denotes all) */
   private final long inputSplitMaxVertices;
   /** Bsp service worker (only use thread-safe methods) */
-  private final BspServiceWorker<I, V, E, M> bspServiceWorker;
+  private final BspServiceWorker<I, V, E> bspServiceWorker;
   /** Filter to select which vertices to keep */
-  private final VertexInputFilter<I, V, E, M> vertexInputFilter;
+  private final VertexInputFilter<I, V, E> vertexInputFilter;
 
   // Metrics
   /** number of vertices loaded meter across all readers */
@@ -85,7 +83,6 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
    *
    * @param vertexInputFormat Vertex input format
    * @param context Context
-   * @param graphState Graph state
    * @param configuration Configuration
    * @param bspServiceWorker service worker
    * @param splitsHandler Handler for input splits
@@ -94,13 +91,12 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
   public VertexInputSplitsCallable(
       VertexInputFormat<I, V, E> vertexInputFormat,
       Mapper<?, ?, ?, ?>.Context context,
-      GraphState<I, V, E, M> graphState,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
-      BspServiceWorker<I, V, E, M> bspServiceWorker,
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
+      BspServiceWorker<I, V, E> bspServiceWorker,
       InputSplitsHandler splitsHandler,
       ZooKeeperExt zooKeeperExt)  {
-    super(context, graphState, configuration, bspServiceWorker,
-        splitsHandler, zooKeeperExt);
+    super(context, configuration, bspServiceWorker, splitsHandler,
+        zooKeeperExt);
     this.vertexInputFormat = vertexInputFormat;
 
     inputSplitMaxVertices = configuration.getInputSplitMaxVertices();
@@ -123,20 +119,16 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
    * maximum number of vertices to be read from an input split.
    *
    * @param inputSplit Input split to process with vertex reader
-   * @param graphState Current graph state
    * @return Vertices and edges loaded from this input split
    * @throws IOException
    * @throws InterruptedException
    */
   @Override
   protected VertexEdgeCount readInputSplit(
-      InputSplit inputSplit,
-      GraphState<I, V, E, M> graphState)
-    throws IOException, InterruptedException {
+      InputSplit inputSplit) throws IOException, InterruptedException {
     VertexReader<I, V, E> vertexReader =
         vertexInputFormat.createVertexReader(inputSplit, context);
-    vertexReader.setConf(
-        (ImmutableClassesGiraphConfiguration<I, V, E, Writable>) configuration);
+    vertexReader.setConf(configuration);
     vertexReader.initialize(inputSplit, context);
 
     long inputSplitVerticesLoaded = 0;
@@ -146,8 +138,7 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
     long inputSplitEdgesLoaded = 0;
 
     while (vertexReader.nextVertex()) {
-      Vertex<I, V, E, M> readerVertex =
-          (Vertex<I, V, E, M>) vertexReader.getCurrentVertex();
+      Vertex<I, V, E> readerVertex = vertexReader.getCurrentVertex();
       if (readerVertex.getId() == null) {
         throw new IllegalArgumentException(
             "readInputSplit: Vertex reader returned a vertex " +
@@ -157,7 +148,6 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
         readerVertex.setValue(configuration.createVertexValue());
       }
       readerVertex.setConf(configuration);
-      readerVertex.setGraphState(graphState);
 
       ++inputSplitVerticesLoaded;
 
@@ -172,7 +162,7 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
 
       PartitionOwner partitionOwner =
           bspServiceWorker.getVertexPartitionOwner(readerVertex.getId());
-      graphState.getWorkerClientRequestProcessor().sendVertexRequest(
+      workerClientRequestProcessor.sendVertexRequest(
           partitionOwner, readerVertex);
       context.progress(); // do this before potential data transfer
       edgesSinceLastUpdate += readerVertex.getNumEdges();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallableFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallableFactory.java b/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallableFactory.java
index cf5e8ad..c9893d2 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallableFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/VertexInputSplitsCallableFactory.java
@@ -19,7 +19,6 @@
 package org.apache.giraph.worker;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.VertexEdgeCount;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.utils.CallableFactory;
@@ -34,21 +33,18 @@ import org.apache.hadoop.mapreduce.Mapper;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class VertexInputSplitsCallableFactory<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     implements CallableFactory<VertexEdgeCount> {
   /** Vertex input format */
   private final VertexInputFormat<I, V, E> vertexInputFormat;
   /** Mapper context. */
   private final Mapper<?, ?, ?, ?>.Context context;
-  /** Graph state. */
-  private final GraphState<I, V, E, M> graphState;
   /** Configuration. */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /** {@link BspServiceWorker} we're running on. */
-  private final BspServiceWorker<I, V, E, M> bspServiceWorker;
+  private final BspServiceWorker<I, V, E> bspServiceWorker;
   /** Handler for input splits */
   private final InputSplitsHandler splitsHandler;
   /** {@link ZooKeeperExt} for this worker. */
@@ -59,7 +55,6 @@ public class VertexInputSplitsCallableFactory<I extends WritableComparable,
    *
    * @param vertexInputFormat Vertex input format
    * @param context Mapper context
-   * @param graphState Graph state
    * @param configuration Configuration
    * @param bspServiceWorker Calling {@link BspServiceWorker}
    * @param splitsHandler Handler for input splits
@@ -68,14 +63,12 @@ public class VertexInputSplitsCallableFactory<I extends WritableComparable,
   public VertexInputSplitsCallableFactory(
       VertexInputFormat<I, V, E> vertexInputFormat,
       Mapper<?, ?, ?, ?>.Context context,
-      GraphState<I, V, E, M> graphState,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
-      BspServiceWorker<I, V, E, M> bspServiceWorker,
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
+      BspServiceWorker<I, V, E> bspServiceWorker,
       InputSplitsHandler splitsHandler,
       ZooKeeperExt zooKeeperExt) {
     this.vertexInputFormat = vertexInputFormat;
     this.context = context;
-    this.graphState = graphState;
     this.configuration = configuration;
     this.bspServiceWorker = bspServiceWorker;
     this.zooKeeperExt = zooKeeperExt;
@@ -83,11 +76,10 @@ public class VertexInputSplitsCallableFactory<I extends WritableComparable,
   }
 
   @Override
-  public InputSplitsCallable<I, V, E, M> newCallable(int threadId) {
-    return new VertexInputSplitsCallable<I, V, E, M>(
+  public InputSplitsCallable<I, V, E> newCallable(int threadId) {
+    return new VertexInputSplitsCallable<I, V, E>(
         vertexInputFormat,
         context,
-        graphState,
         configuration,
         bspServiceWorker,
         splitsHandler,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
index 9a8a8b8..9bfd7b5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerAggregatorHandler.java
@@ -64,7 +64,7 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
   private Map<String, Aggregator<Writable>> currentAggregatorMap =
       Maps.newHashMap();
   /** Service worker */
-  private final CentralizedServiceWorker<?, ?, ?, ?> serviceWorker;
+  private final CentralizedServiceWorker<?, ?, ?> serviceWorker;
   /** Progressable for reporting progress */
   private final Progressable progressable;
   /** How big a single aggregator request can be */
@@ -80,7 +80,7 @@ public class WorkerAggregatorHandler implements WorkerThreadAggregatorUsage {
    * @param progressable  Progressable for reporting progress
    */
   public WorkerAggregatorHandler(
-      CentralizedServiceWorker<?, ?, ?, ?> serviceWorker,
+      CentralizedServiceWorker<?, ?, ?> serviceWorker,
       ImmutableClassesGiraphConfiguration conf,
       Progressable progressable) {
     this.serviceWorker = serviceWorker;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
index d3ffaea..729ba14 100644
--- a/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
+++ b/giraph-core/src/main/java/org/apache/giraph/worker/WorkerContext.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.mapreduce.Mapper;
 public abstract class WorkerContext implements WorkerAggregatorUsage {
   /** Global graph state */
   private GraphState graphState;
+  /** Worker aggregator usage */
+  private WorkerAggregatorUsage workerAggregatorUsage;
 
   /**
    * Set the graph state.
@@ -41,6 +43,16 @@ public abstract class WorkerContext implements WorkerAggregatorUsage {
   }
 
   /**
+   * Set worker aggregator usage
+   *
+   * @param workerAggregatorUsage Worker aggregator usage
+   */
+  public void setWorkerAggregatorUsage(
+      WorkerAggregatorUsage workerAggregatorUsage) {
+    this.workerAggregatorUsage = workerAggregatorUsage;
+  }
+
+  /**
    * Initialize the WorkerContext.
    * This method is executed once on each Worker before the first
    * superstep starts.
@@ -112,11 +124,11 @@ public abstract class WorkerContext implements WorkerAggregatorUsage {
 
   @Override
   public <A extends Writable> void aggregate(String name, A value) {
-    graphState.getWorkerAggregatorUsage().aggregate(name, value);
+    workerAggregatorUsage.aggregate(name, value);
   }
 
   @Override
   public <A extends Writable> A getAggregatedValue(String name) {
-    return graphState.getWorkerAggregatorUsage().<A>getAggregatedValue(name);
+    return workerAggregatorUsage.<A>getAggregatedValue(name);
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
index 91b842a..e771e36 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/ConnectionTest.java
@@ -25,8 +25,8 @@ import org.apache.giraph.comm.netty.handler.RequestServerHandler;
 import org.apache.giraph.comm.netty.handler.WorkerRequestServerHandler;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.IntNoOpComputation;
 import org.apache.giraph.utils.MockUtils;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.mapreduce.Mapper.Context;
@@ -46,17 +46,10 @@ public class ConnectionTest {
   /** Class configuration */
   private ImmutableClassesGiraphConfiguration conf;
 
-  public static class IntVertex extends Vertex<IntWritable,
-            IntWritable, IntWritable, IntWritable> {
-    @Override
-    public void compute(Iterable<IntWritable> messages) throws IOException {
-    }
-  }
-
   @Before
   public void setUp() {
     GiraphConfiguration tmpConfig = new GiraphConfiguration();
-    tmpConfig.setVertexClass(IntVertex.class);
+    tmpConfig.setComputationClass(IntNoOpComputation.class);
     conf = new ImmutableClassesGiraphConfiguration(tmpConfig);
   }
 
@@ -71,7 +64,7 @@ public class ConnectionTest {
     Context context = mock(Context.class);
     when(context.getConfiguration()).thenReturn(conf);
 
-    ServerData<IntWritable, IntWritable, IntWritable, IntWritable> serverData =
+    ServerData<IntWritable, IntWritable, IntWritable> serverData =
         MockUtils.createNewServerData(conf, context);
     WorkerInfo workerInfo = new WorkerInfo();
     NettyServer server =
@@ -100,7 +93,7 @@ public class ConnectionTest {
     Context context = mock(Context.class);
     when(context.getConfiguration()).thenReturn(conf);
 
-    ServerData<IntWritable, IntWritable, IntWritable, IntWritable> serverData =
+    ServerData<IntWritable, IntWritable, IntWritable> serverData =
         MockUtils.createNewServerData(conf, context);
    RequestServerHandler.Factory requestServerHandlerFactory =
        new WorkerRequestServerHandler.Factory(serverData);
@@ -150,7 +143,7 @@ public class ConnectionTest {
     Context context = mock(Context.class);
     when(context.getConfiguration()).thenReturn(conf);
 
-    ServerData<IntWritable, IntWritable, IntWritable, IntWritable> serverData =
+    ServerData<IntWritable, IntWritable, IntWritable> serverData =
         MockUtils.createNewServerData(conf, context);
     WorkerInfo workerInfo = new WorkerInfo();
     NettyServer server = new NettyServer(conf,

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
index 58aa7d1..5c69161 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/RequestFailureTest.java
@@ -26,8 +26,8 @@ import org.apache.giraph.comm.requests.WritableRequest;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
+import org.apache.giraph.utils.IntNoOpComputation;
 import org.apache.giraph.utils.MockUtils;
 import org.apache.giraph.utils.PairList;
 import org.apache.giraph.worker.WorkerInfo;
@@ -51,7 +51,7 @@ public class RequestFailureTest {
   /** Configuration */
   private ImmutableClassesGiraphConfiguration conf;
   /** Server data */
-  private ServerData<IntWritable, IntWritable, IntWritable, IntWritable>
+  private ServerData<IntWritable, IntWritable, IntWritable>
   serverData;
   /** Server */
   private NettyServer server;
@@ -60,21 +60,11 @@ public class RequestFailureTest {
   /** Mock context */
   private Context context;
 
-  /**
-   * Only for testing.
-   */
-  public static class TestVertex extends Vertex<IntWritable,
-        IntWritable, IntWritable, IntWritable> {
-    @Override
-    public void compute(Iterable<IntWritable> messages) throws IOException {
-    }
-  }
-
   @Before
   public void setUp() throws IOException {
     // Setup the conf
     GiraphConfiguration tmpConf = new GiraphConfiguration();
-    tmpConf.setVertexClass(TestVertex.class);
+    tmpConf.setComputationClass(IntNoOpComputation.class);
     conf = new ImmutableClassesGiraphConfiguration(tmpConf);
 
     context = mock(Context.class);
@@ -91,7 +81,8 @@ public class RequestFailureTest {
     dataToSend.initialize();
     ByteArrayVertexIdMessages<IntWritable,
             IntWritable> vertexIdMessages =
-        new ByteArrayVertexIdMessages<IntWritable, IntWritable>();
+        new ByteArrayVertexIdMessages<IntWritable, IntWritable>(
+            IntWritable.class);
     vertexIdMessages.setConf(conf);
     vertexIdMessages.initialize();
     dataToSend.add(partitionId, vertexIdMessages);
@@ -117,7 +108,8 @@ public class RequestFailureTest {
     for (IntWritable vertexId : vertices) {
       keySum += vertexId.get();
       Iterable<IntWritable> messages =
-          serverData.getIncomingMessageStore().getVertexMessages(vertexId);
+          serverData.<IntWritable>getIncomingMessageStore().getVertexMessages(
+              vertexId);
       synchronized (messages) {
         for (IntWritable message : messages) {
           messageSum += message.get();

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
index f1f8e26..7016572 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
@@ -35,6 +35,7 @@ import org.apache.giraph.metrics.GiraphMetrics;
 import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionStore;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
+import org.apache.giraph.utils.IntNoOpComputation;
 import org.apache.giraph.utils.MockUtils;
 import org.apache.giraph.utils.PairList;
 import org.apache.giraph.worker.WorkerInfo;
@@ -63,8 +64,7 @@ public class RequestTest {
   /** Configuration */
   private ImmutableClassesGiraphConfiguration conf;
   /** Server data */
-  private ServerData<IntWritable, IntWritable, IntWritable, IntWritable>
-  serverData;
+  private ServerData<IntWritable, IntWritable, IntWritable> serverData;
   /** Server */
   private NettyServer server;
   /** Client */
@@ -72,21 +72,11 @@ public class RequestTest {
   /** Worker info */
   private WorkerInfo workerInfo;
 
-  /**
-   * Only for testing.
-   */
-  public static class TestVertex extends Vertex<IntWritable,
-      IntWritable, IntWritable, IntWritable> {
-    @Override
-    public void compute(Iterable<IntWritable> messages) throws IOException {
-    }
-  }
-
   @Before
   public void setUp() throws IOException {
     // Setup the conf
     GiraphConfiguration tmpConf = new GiraphConfiguration();
-    GiraphConstants.VERTEX_CLASS.set(tmpConf, TestVertex.class);
+    GiraphConstants.COMPUTATION_CLASS.set(tmpConf, IntNoOpComputation.class);
     conf = new ImmutableClassesGiraphConfiguration(tmpConf);
 
     @SuppressWarnings("rawtypes")
@@ -110,7 +100,7 @@ public class RequestTest {
   public void sendVertexPartition() throws IOException {
     // Data to send
     int partitionId = 13;
-    Partition<IntWritable, IntWritable, IntWritable, IntWritable> partition =
+    Partition<IntWritable, IntWritable, IntWritable> partition =
         conf.createPartition(partitionId, null);
     for (int i = 0; i < 10; ++i) {
       Vertex vertex = conf.createVertex();
@@ -119,10 +109,8 @@ public class RequestTest {
     }
 
     // Send the request
-    SendVertexRequest<IntWritable, IntWritable, IntWritable,
-    IntWritable> request =
-      new SendVertexRequest<IntWritable, IntWritable,
-      IntWritable, IntWritable>(partition);
+    SendVertexRequest<IntWritable, IntWritable, IntWritable> request =
+      new SendVertexRequest<IntWritable, IntWritable, IntWritable>(partition);
     client.sendWritableRequest(workerInfo.getTaskId(), request);
     client.waitAllRequests();
 
@@ -131,15 +119,13 @@ public class RequestTest {
     server.stop();
 
     // Check the output
-    PartitionStore<IntWritable, IntWritable,
-        IntWritable, IntWritable> partitionStore =
+    PartitionStore<IntWritable, IntWritable, IntWritable> partitionStore =
         serverData.getPartitionStore();
     assertTrue(partitionStore.hasPartition(partitionId));
     int total = 0;
-    Partition<IntWritable, IntWritable, IntWritable, IntWritable> partition2 =
+    Partition<IntWritable, IntWritable, IntWritable> partition2 =
         partitionStore.getPartition(partitionId);	
-    for (Vertex<IntWritable, IntWritable,
-        IntWritable, IntWritable> vertex : partition2) {
+    for (Vertex<IntWritable, IntWritable, IntWritable> vertex : partition2) {
       total += vertex.getId().get();
     }
     partitionStore.putPartition(partition2);
@@ -158,7 +144,8 @@ public class RequestTest {
     int partitionId = 0;
     ByteArrayVertexIdMessages<IntWritable,
             IntWritable> vertexIdMessages =
-        new ByteArrayVertexIdMessages<IntWritable, IntWritable>();
+        new ByteArrayVertexIdMessages<IntWritable, IntWritable>(
+            IntWritable.class);
     vertexIdMessages.setConf(conf);
     vertexIdMessages.initialize();
     dataToSend.add(partitionId, vertexIdMessages);
@@ -187,7 +174,8 @@ public class RequestTest {
     for (IntWritable vertexId : vertices) {
       keySum += vertexId.get();
       Iterable<IntWritable> messages =
-          serverData.getIncomingMessageStore().getVertexMessages(vertexId);
+          serverData.<IntWritable>getIncomingMessageStore().getVertexMessages(
+              vertexId);
       synchronized (messages) {
         for (IntWritable message : messages) {
           messageSum += message.get();
@@ -203,12 +191,11 @@ public class RequestTest {
     // Data to send
     int partitionId = 19;
     Map<IntWritable, VertexMutations<IntWritable, IntWritable,
-    IntWritable, IntWritable>> vertexIdMutations =
+    IntWritable>> vertexIdMutations =
         Maps.newHashMap();
     for (int i = 0; i < 11; ++i) {
-      VertexMutations<IntWritable, IntWritable, IntWritable, IntWritable>
-      mutations = new VertexMutations<IntWritable, IntWritable,
-          IntWritable, IntWritable>();
+      VertexMutations<IntWritable, IntWritable, IntWritable> mutations =
+          new VertexMutations<IntWritable, IntWritable, IntWritable>();
       for (int j = 0; j < 3; ++j) {
         Vertex vertex = conf.createVertex();
         vertex.initialize(new IntWritable(i), new IntWritable(j));
@@ -229,10 +216,9 @@ public class RequestTest {
     }
 
     // Send the request
-    SendPartitionMutationsRequest<IntWritable, IntWritable, IntWritable,
-    IntWritable> request =
-      new SendPartitionMutationsRequest<IntWritable, IntWritable,
-      IntWritable, IntWritable>(partitionId, vertexIdMutations);
+    SendPartitionMutationsRequest<IntWritable, IntWritable, IntWritable>
+        request = new SendPartitionMutationsRequest<IntWritable, IntWritable,
+        IntWritable>(partitionId, vertexIdMutations);
     GiraphMetrics.init(conf);
     client.sendWritableRequest(workerInfo.getTaskId(), request);
     client.waitAllRequests();
@@ -243,16 +229,16 @@ public class RequestTest {
 
     // Check the output
     ConcurrentHashMap<IntWritable, VertexMutations<IntWritable, IntWritable,
-    IntWritable, IntWritable>> inVertexIdMutations =
+    IntWritable>> inVertexIdMutations =
         serverData.getVertexMutations();
     int keySum = 0;
     for (Entry<IntWritable, VertexMutations<IntWritable, IntWritable,
-        IntWritable, IntWritable>> entry :
+        IntWritable>> entry :
           inVertexIdMutations.entrySet()) {
       synchronized (entry.getValue()) {
         keySum += entry.getKey().get();
         int vertexValueSum = 0;
-        for (Vertex<IntWritable, IntWritable, IntWritable, IntWritable>
+        for (Vertex<IntWritable, IntWritable, IntWritable>
         vertex : entry.getValue().getAddedVertexList()) {
           vertexValueSum += vertex.getValue().get();
         }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
index c27156f..c026cf8 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/SaslConnectionTest.java
@@ -25,7 +25,7 @@ import org.apache.giraph.comm.netty.handler.WorkerRequestServerHandler;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.utils.IntNoOpComputation;
 import org.apache.giraph.utils.MockUtils;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.IntWritable;
@@ -48,17 +48,10 @@ public class SaslConnectionTest {
   /** Class configuration */
   private ImmutableClassesGiraphConfiguration conf;
 
-  public static class IntVertex extends Vertex<IntWritable,
-          IntWritable, IntWritable, IntWritable> {
-    @Override
-    public void compute(Iterable<IntWritable> messages) throws IOException {
-    }
-  }
-
   @Before
   public void setUp() {
     GiraphConfiguration tmpConfig = new GiraphConfiguration();
-    tmpConfig.setVertexClass(IntVertex.class);
+    tmpConfig.setComputationClass(IntNoOpComputation.class);
     GiraphConstants.AUTHENTICATE.set(tmpConfig, true);
     conf = new ImmutableClassesGiraphConfiguration(tmpConfig);
   }
@@ -74,7 +67,7 @@ public class SaslConnectionTest {
     Context context = mock(Context.class);
     when(context.getConfiguration()).thenReturn(conf);
 
-    ServerData<IntWritable, IntWritable, IntWritable, IntWritable> serverData =
+    ServerData<IntWritable, IntWritable, IntWritable> serverData =
         MockUtils.createNewServerData(conf, context);
 
     SaslServerHandler.Factory mockedSaslServerFactory =


[11/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
index 3cd1175..97c8a35 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
@@ -23,6 +23,7 @@ import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
 import org.apache.giraph.utils.ExtendedDataOutput;
+import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.giraph.utils.RepresentativeByteArrayIterable;
 import org.apache.giraph.utils.RepresentativeByteArrayIterator;
 import org.apache.giraph.utils.VertexIdIterator;
@@ -47,13 +48,15 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
   /**
    * Constructor
    *
+   * @param messageClass Message class held in the store
    * @param service Service worker
    * @param config Hadoop configuration
    */
   public ByteArrayMessagesPerVertexStore(
-      CentralizedServiceWorker<I, ?, ?, M> service,
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config) {
-    super(service, config);
+      Class<M> messageClass,
+      CentralizedServiceWorker<I, ?, ?> service,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
+    super(messageClass, service, config);
   }
 
   /**
@@ -142,7 +145,7 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
 
     @Override
     protected M createWritable() {
-      return config.createMessageValue();
+      return ReflectionUtils.newInstance(messageClass);
     }
   }
 
@@ -175,7 +178,7 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
 
     @Override
     protected M createWritable() {
-      return config.createMessageValue();
+      return ReflectionUtils.newInstance(messageClass);
     }
   }
 
@@ -222,8 +225,8 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
    */
   public static <I extends WritableComparable, M extends Writable>
   MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> newFactory(
-      CentralizedServiceWorker<I, ?, ?, M> service,
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config) {
+      CentralizedServiceWorker<I, ?, ?> service,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
     return new Factory<I, M>(service, config);
   }
 
@@ -272,23 +275,23 @@ public class ByteArrayMessagesPerVertexStore<I extends WritableComparable,
   private static class Factory<I extends WritableComparable, M extends Writable>
       implements MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> {
     /** Service worker */
-    private final CentralizedServiceWorker<I, ?, ?, M> service;
+    private final CentralizedServiceWorker<I, ?, ?> service;
     /** Hadoop configuration */
-    private final ImmutableClassesGiraphConfiguration<I, ?, ?, M> config;
+    private final ImmutableClassesGiraphConfiguration<I, ?, ?> config;
 
     /**
      * @param service Worker service
      * @param config  Hadoop configuration
      */
-    public Factory(CentralizedServiceWorker<I, ?, ?, M> service,
-        ImmutableClassesGiraphConfiguration<I, ?, ?, M> config) {
+    public Factory(CentralizedServiceWorker<I, ?, ?> service,
+        ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
       this.service = service;
       this.config = config;
     }
 
     @Override
-    public MessageStoreByPartition<I, M> newStore() {
-      return new ByteArrayMessagesPerVertexStore(service, config);
+    public MessageStoreByPartition<I, M> newStore(Class<M> messageClass) {
+      return new ByteArrayMessagesPerVertexStore(messageClass, service, config);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
index 26abb94..2712edd 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
@@ -35,6 +35,7 @@ import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.ExtendedDataOutput;
+import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.giraph.utils.RepresentativeByteArrayIterable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -48,6 +49,8 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public class DiskBackedMessageStore<I extends WritableComparable,
     M extends Writable> implements FlushableMessageStore<I, M> {
+  /** Message class */
+  private final Class<M> messageClass;
   /**
    * In-memory message map (must be sorted to insure that the ids are
    * ordered)
@@ -55,7 +58,7 @@ public class DiskBackedMessageStore<I extends WritableComparable,
   private volatile ConcurrentNavigableMap<I, ExtendedDataOutput>
   inMemoryMessages;
   /** Hadoop configuration */
-  private final ImmutableClassesGiraphConfiguration<I, ?, ?, M> config;
+  private final ImmutableClassesGiraphConfiguration<I, ?, ?> config;
   /** Counter for number of messages in-memory */
   private final AtomicInteger numberOfMessagesInMemory;
   /** To keep vertex ids which we have messages for */
@@ -71,13 +74,16 @@ public class DiskBackedMessageStore<I extends WritableComparable,
   /**
    * Constructor.
    *
+   * @param messageClass     Message class held in the store
    * @param config           Hadoop configuration
    * @param fileStoreFactory Factory for creating file stores when flushing
    */
   public DiskBackedMessageStore(
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config,
+      Class<M> messageClass,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config,
       MessageStoreFactory<I, M, BasicMessageStore<I, M>> fileStoreFactory) {
     inMemoryMessages = new ConcurrentSkipListMap<I, ExtendedDataOutput>();
+    this.messageClass = messageClass;
     this.config = config;
     numberOfMessagesInMemory = new AtomicInteger(0);
     destinationVertices =
@@ -154,7 +160,7 @@ public class DiskBackedMessageStore<I extends WritableComparable,
 
     @Override
     protected M createWritable() {
-      return config.createMessageValue();
+      return ReflectionUtils.newInstance(messageClass);
     }
   }
 
@@ -289,7 +295,8 @@ public class DiskBackedMessageStore<I extends WritableComparable,
     } finally {
       rwLock.writeLock().unlock();
     }
-    BasicMessageStore<I, M> fileStore = fileStoreFactory.newStore();
+    BasicMessageStore<I, M> fileStore =
+        fileStoreFactory.newStore(messageClass);
     fileStore.addMessages(new TemporaryMessageStore(messagesToFlush));
 
     synchronized (fileStores) {
@@ -351,7 +358,8 @@ public class DiskBackedMessageStore<I extends WritableComparable,
     // read file stores
     int numFileStores = in.readInt();
     for (int s = 0; s < numFileStores; s++) {
-      BasicMessageStore<I, M> fileStore = fileStoreFactory.newStore();
+      BasicMessageStore<I, M> fileStore =
+          fileStoreFactory.newStore(messageClass);
       fileStore.readFields(in);
       fileStores.add(fileStore);
     }
@@ -370,7 +378,7 @@ public class DiskBackedMessageStore<I extends WritableComparable,
    */
   public static <I extends WritableComparable, M extends Writable>
   MessageStoreFactory<I, M, FlushableMessageStore<I, M>> newFactory(
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config,
       MessageStoreFactory<I, M, BasicMessageStore<I, M>> fileStoreFactory) {
     return new Factory<I, M>(config, fileStoreFactory);
   }
@@ -402,8 +410,9 @@ public class DiskBackedMessageStore<I extends WritableComparable,
     }
 
     @Override
-    public FlushableMessageStore<I, M> newStore() {
-      return new DiskBackedMessageStore<I, M>(config, fileStoreFactory);
+    public FlushableMessageStore<I, M> newStore(Class<M> messageClass) {
+      return new DiskBackedMessageStore<I, M>(messageClass, config,
+          fileStoreFactory);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStoreByPartition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStoreByPartition.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStoreByPartition.java
index 02bfb1f..4a28949 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStoreByPartition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStoreByPartition.java
@@ -47,8 +47,10 @@ import java.util.concurrent.ConcurrentMap;
 public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
     V extends Writable, E extends Writable, M extends Writable> implements
     MessageStoreByPartition<I, M> {
+  /** Message class */
+  private final Class<M> messageClass;
   /** Service worker */
-  private final CentralizedServiceWorker<I, V, E, M> service;
+  private final CentralizedServiceWorker<I, V, E> service;
   /** Number of messages to keep in memory */
   private final int maxNumberOfMessagesInMemory;
   /** Factory for creating file stores when flushing */
@@ -57,18 +59,20 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
   /** Map from partition id to its message store */
   private final
   ConcurrentMap<Integer, FlushableMessageStore<I, M>> partitionMessageStores;
-
   /**
+   * @param messageClass                Message class held in the store
    * @param service                     Service worker
    * @param maxNumberOfMessagesInMemory Number of messages to keep in memory
    * @param fileStoreFactory            Factory for creating file stores
    *                                    when flushing
    */
   public DiskBackedMessageStoreByPartition(
-      CentralizedServiceWorker<I, V, E, M> service,
+      Class<M> messageClass,
+      CentralizedServiceWorker<I, V, E> service,
       int maxNumberOfMessagesInMemory,
       MessageStoreFactory<I, M, FlushableMessageStore<I,
           M>> fileStoreFactory) {
+    this.messageClass = messageClass;
     this.service = service;
     this.maxNumberOfMessagesInMemory = maxNumberOfMessagesInMemory;
     this.fileStoreFactory = fileStoreFactory;
@@ -267,7 +271,7 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
     if (messageStore != null) {
       return messageStore;
     }
-    messageStore = fileStoreFactory.newStore();
+    messageStore = fileStoreFactory.newStore(messageClass);
     FlushableMessageStore<I, M> store =
         partitionMessageStores.putIfAbsent(partitionId, messageStore);
     return (store == null) ? messageStore : store;
@@ -298,7 +302,8 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
   public void readFieldsForPartition(DataInput in,
       int partitionId) throws IOException {
     if (in.readBoolean()) {
-      FlushableMessageStore<I, M> messageStore = fileStoreFactory.newStore();
+      FlushableMessageStore<I, M> messageStore =
+          fileStoreFactory.newStore(messageClass);
       messageStore.readFields(in);
       partitionMessageStores.put(partitionId, messageStore);
     }
@@ -309,7 +314,8 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
     int numStores = in.readInt();
     for (int s = 0; s < numStores; s++) {
       int partitionId = in.readInt();
-      FlushableMessageStore<I, M> messageStore = fileStoreFactory.newStore();
+      FlushableMessageStore<I, M> messageStore =
+          fileStoreFactory.newStore(messageClass);
       messageStore.readFields(in);
       partitionMessageStores.put(partitionId, messageStore);
     }
@@ -332,7 +338,7 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
   public static <I extends WritableComparable, V extends Writable,
       E extends Writable, M extends Writable>
   MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> newFactory(
-      CentralizedServiceWorker<I, V, E, M> service,
+      CentralizedServiceWorker<I, V, E> service,
       int maxMessagesInMemory,
       MessageStoreFactory<I, M, FlushableMessageStore<I, M>>
           fileStoreFactory) {
@@ -352,7 +358,7 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
       V extends Writable, E extends Writable, M extends Writable>
       implements MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> {
     /** Service worker */
-    private final CentralizedServiceWorker<I, V, E, M> service;
+    private final CentralizedServiceWorker<I, V, E> service;
     /** Number of messages to keep in memory */
     private final int maxMessagesInMemory;
     /** Factory for creating file stores when flushing */
@@ -365,7 +371,7 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
      * @param fileStoreFactory    Factory for creating file stores when
      *                            flushing
      */
-    public Factory(CentralizedServiceWorker<I, V, E, M> service,
+    public Factory(CentralizedServiceWorker<I, V, E> service,
         int maxMessagesInMemory,
         MessageStoreFactory<I, M, FlushableMessageStore<I, M>>
             fileStoreFactory) {
@@ -375,9 +381,9 @@ public class DiskBackedMessageStoreByPartition<I extends WritableComparable,
     }
 
     @Override
-    public MessageStoreByPartition<I, M> newStore() {
-      return new DiskBackedMessageStoreByPartition<I, V, E, M>(service,
-          maxMessagesInMemory, fileStoreFactory);
+    public MessageStoreByPartition<I, M> newStore(Class<M> messageClass) {
+      return new DiskBackedMessageStoreByPartition<I, V, E, M>(messageClass,
+          service, maxMessagesInMemory, fileStoreFactory);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
new file mode 100644
index 0000000..9086d78
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/InMemoryMessageStoreFactory.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.comm.messages;
+
+import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.log4j.Logger;
+
+/**
+ * Message store factory which produces message stores which hold all
+ * messages in memory. Depending on whether or not combiner is currently used,
+ * this factory creates {@link OneMessagePerVertexStore} or
+ * {@link ByteArrayMessagesPerVertexStore}
+ *
+ * @param <I> Vertex id
+ * @param <M> Message data
+ */
+public class InMemoryMessageStoreFactory<I extends WritableComparable,
+    M extends Writable>
+    implements MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> {
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(InMemoryMessageStoreFactory.class);
+
+  /** Service worker */
+  private final CentralizedServiceWorker<I, ?, ?> service;
+  /** Hadoop configuration */
+  private final ImmutableClassesGiraphConfiguration<I, ?, ?> conf;
+
+  /**
+   * @param service Worker service
+   * @param conf    Configuration
+   */
+  public InMemoryMessageStoreFactory(CentralizedServiceWorker<I, ?, ?> service,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> conf) {
+    this.service = service;
+    this.conf = conf;
+  }
+
+  @Override
+  public MessageStoreByPartition<I, M> newStore(Class<M> messageClass) {
+    if (conf.useCombiner()) {
+      if (LOG.isInfoEnabled()) {
+        LOG.info("newStore: " +
+            "Using OneMessagePerVertexStore with " + conf.getCombinerClass());
+      }
+      return new OneMessagePerVertexStore<I, M>(
+          messageClass, service, conf.<M>createCombiner(), conf);
+    } else {
+      if (LOG.isInfoEnabled()) {
+        LOG.info("newStore: " +
+            "Using ByteArrayMessagesPerVertexStore since there is no combiner");
+      }
+      return new ByteArrayMessagesPerVertexStore<I, M>(
+          messageClass, service, conf);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStoreFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStoreFactory.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStoreFactory.java
index 3c13c47..dec9a92 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStoreFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/MessageStoreFactory.java
@@ -31,9 +31,13 @@ import org.apache.hadoop.io.WritableComparable;
 public interface MessageStoreFactory<I extends WritableComparable,
     M extends Writable, S extends BasicMessageStore<I, M>> {
   /**
-   * Creates new message store
+   * Creates new message store.
    *
+   * Note: Combiner class in Configuration can be changed,
+   * this method should return MessageStore which uses current combiner
+   *
+   * @param messageClass Message class held in the store
    * @return New message store
    */
-  S newStore();
+  S newStore(Class<M> messageClass);
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
index 7db0266..8710dac 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
@@ -28,6 +28,7 @@ import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
+import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -45,14 +46,17 @@ public class OneMessagePerVertexStore<I extends WritableComparable,
   private final Combiner<I, M> combiner;
 
   /**
+   * @param messageClass Message class held in the store
    * @param service  Service worker
    * @param combiner Combiner for messages
    * @param config   Hadoop configuration
    */
-  OneMessagePerVertexStore(CentralizedServiceWorker<I, ?, ?, M> service,
+  OneMessagePerVertexStore(
+      Class<M> messageClass,
+      CentralizedServiceWorker<I, ?, ?> service,
       Combiner<I, M> combiner,
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config) {
-    super(service, config);
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
+    super(messageClass, service, config);
     this.combiner = combiner;
   }
 
@@ -140,7 +144,7 @@ public class OneMessagePerVertexStore<I extends WritableComparable,
 
   @Override
   protected M readFieldsForMessages(DataInput in) throws IOException {
-    M message = config.createMessageValue();
+    M message = ReflectionUtils.newInstance(messageClass);
     message.readFields(in);
     return message;
   }
@@ -177,8 +181,8 @@ public class OneMessagePerVertexStore<I extends WritableComparable,
    */
   public static <I extends WritableComparable, M extends Writable>
   MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> newFactory(
-      CentralizedServiceWorker<I, ?, ?, M> service,
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config) {
+      CentralizedServiceWorker<I, ?, ?> service,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
     return new Factory<I, M>(service, config);
   }
 
@@ -192,26 +196,24 @@ public class OneMessagePerVertexStore<I extends WritableComparable,
       M extends Writable>
       implements MessageStoreFactory<I, M, MessageStoreByPartition<I, M>> {
     /** Service worker */
-    private final CentralizedServiceWorker<I, ?, ?, M> service;
+    private final CentralizedServiceWorker<I, ?, ?> service;
     /** Hadoop configuration */
-    private final ImmutableClassesGiraphConfiguration<I, ?, ?, M> config;
-    /** Combiner for messages */
-    private final Combiner<I, M> combiner;
+    private final ImmutableClassesGiraphConfiguration<I, ?, ?> config;
 
     /**
      * @param service Worker service
      * @param config  Hadoop configuration
      */
-    public Factory(CentralizedServiceWorker<I, ?, ?, M> service,
-        ImmutableClassesGiraphConfiguration<I, ?, ?, M> config) {
+    public Factory(CentralizedServiceWorker<I, ?, ?> service,
+        ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
       this.service = service;
       this.config = config;
-      combiner = config.createCombiner();
     }
 
     @Override
-    public MessageStoreByPartition<I, M> newStore() {
-      return new OneMessagePerVertexStore<I, M>(service, combiner, config);
+    public MessageStoreByPartition<I, M> newStore(Class<M> messageClass) {
+      return new OneMessagePerVertexStore<I, M>(messageClass, service,
+          config.<M>createCombiner(), config);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
index 3fe4430..f0a8f6d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
@@ -21,6 +21,7 @@ package org.apache.giraph.comm.messages;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.EmptyIterable;
+import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
@@ -63,10 +64,12 @@ public class SequentialFileMessageStore<I extends WritableComparable,
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(SequentialFileMessageStore.class);
+  /** Message class */
+  private final Class<M> messageClass;
   /** File in which we store data */
   private final File file;
   /** Configuration which we need for reading data */
-  private final ImmutableClassesGiraphConfiguration<I, ?, ?, M> config;
+  private final ImmutableClassesGiraphConfiguration<I, ?, ?> config;
   /** Buffer size to use when reading and writing files */
   private final int bufferSize;
   /** File input stream */
@@ -79,15 +82,18 @@ public class SequentialFileMessageStore<I extends WritableComparable,
   /**
    * Stores message on the disk.
    *
-   * @param config     Configuration used later for reading
-   * @param bufferSize Buffer size to use when reading and writing
-   * @param fileName   File in which we want to store messages
+   * @param messageClass Message class held in the store
+   * @param config       Configuration used later for reading
+   * @param bufferSize   Buffer size to use when reading and writing
+   * @param fileName     File in which we want to store messages
    * @throws IOException
    */
   public SequentialFileMessageStore(
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config,
+      Class<M> messageClass,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config,
       int bufferSize,
       String fileName) {
+    this.messageClass = messageClass;
     this.config = config;
     this.bufferSize = bufferSize;
     file = new File(fileName);
@@ -295,7 +301,7 @@ public class SequentialFileMessageStore<I extends WritableComparable,
     int messagesSize = in.readInt();
     List<M> messages = Lists.newArrayListWithCapacity(messagesSize);
     for (int i = 0; i < messagesSize; i++) {
-      M message = config.createMessageValue();
+      M message = ReflectionUtils.newInstance(messageClass);
       try {
         message.readFields(in);
       } catch (IOException e) {
@@ -394,12 +400,12 @@ public class SequentialFileMessageStore<I extends WritableComparable,
     }
 
     @Override
-    public BasicMessageStore<I, M> newStore() {
+    public BasicMessageStore<I, M> newStore(Class<M> messageClass) {
       int idx = Math.abs(storeCounter.getAndIncrement());
       String fileName =
           directories[idx % directories.length] + "messages-" + idx;
-      return new SequentialFileMessageStore<I, M>(config, bufferSize,
-          fileName);
+      return new SequentialFileMessageStore<I, M>(messageClass, config,
+          bufferSize, fileName);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java b/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
index fc6560d..1a91dfb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
@@ -45,22 +45,27 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public abstract class SimpleMessageStore<I extends WritableComparable,
     M extends Writable, T> implements MessageStoreByPartition<I, M>  {
+  /** Message class */
+  protected final Class<M> messageClass;
   /** Service worker */
-  protected final CentralizedServiceWorker<I, ?, ?, M> service;
+  protected final CentralizedServiceWorker<I, ?, ?> service;
   /** Map from partition id to map from vertex id to messages for that vertex */
   protected final ConcurrentMap<Integer, ConcurrentMap<I, T>> map;
   /** Giraph configuration */
-  protected final ImmutableClassesGiraphConfiguration<I, ?, ?, M> config;
+  protected final ImmutableClassesGiraphConfiguration<I, ?, ?> config;
 
   /**
    * Constructor
    *
+   * @param messageClass Message class held in the store
    * @param service Service worker
    * @param config Giraph configuration
    */
   public SimpleMessageStore(
-      CentralizedServiceWorker<I, ?, ?, M> service,
-      ImmutableClassesGiraphConfiguration<I, ?, ?, M> config) {
+      Class<M> messageClass,
+      CentralizedServiceWorker<I, ?, ?> service,
+      ImmutableClassesGiraphConfiguration<I, ?, ?> config) {
+    this.messageClass = messageClass;
     this.service = service;
     this.config = config;
     map = new MapMaker().concurrencyLevel(

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
index 319f41a..c982209 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
@@ -39,7 +39,7 @@ public class NettyMasterClient implements MasterClient {
   /** Netty client that does the actual I/O */
   private final NettyClient nettyClient;
   /** Worker information for current superstep */
-  private CentralizedServiceMaster<?, ?, ?, ?> service;
+  private CentralizedServiceMaster<?, ?, ?> service;
   /** Cached map of partition ids to serialized aggregator data */
   private final SendAggregatorCache sendAggregatorCache =
       new SendAggregatorCache();
@@ -57,7 +57,7 @@ public class NettyMasterClient implements MasterClient {
    */
   public NettyMasterClient(Mapper<?, ?, ?, ?>.Context context,
                            ImmutableClassesGiraphConfiguration configuration,
-                           CentralizedServiceMaster<?, ?, ?, ?> service) {
+                           CentralizedServiceMaster<?, ?, ?> service) {
     this.nettyClient =
         new NettyClient(context, configuration, service.getMasterInfo());
     this.service = service;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
index 1f04bcf..cb36c3e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
@@ -41,7 +41,7 @@ public class NettyMasterServer implements MasterServer {
    * @param progressable Progressable for reporting progress
    */
   public NettyMasterServer(ImmutableClassesGiraphConfiguration conf,
-      CentralizedServiceMaster<?, ?, ?, ?> service,
+      CentralizedServiceMaster<?, ?, ?> service,
       Progressable progressable) {
     nettyServer = new NettyServer(conf,
         new MasterRequestServerHandler.Factory(service.getAggregatorHandler()),

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
index d1cce64..798e09c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
@@ -41,9 +41,9 @@ public class NettyWorkerAggregatorRequestProcessor
   /** Progressable used to report progress */
   private final Progressable progressable;
   /** NettyClient that could be shared among one or more instances */
-  private final WorkerClient<?, ?, ?, ?> workerClient;
+  private final WorkerClient<?, ?, ?> workerClient;
   /** Service worker */
-  private final CentralizedServiceWorker<?, ?, ?, ?> serviceWorker;
+  private final CentralizedServiceWorker<?, ?, ?> serviceWorker;
   /** Cached map of partition ids to serialized aggregator data */
   private final SendAggregatedValueCache sendAggregatedValueCache =
       new SendAggregatedValueCache();
@@ -59,8 +59,8 @@ public class NettyWorkerAggregatorRequestProcessor
    */
   public NettyWorkerAggregatorRequestProcessor(
       Progressable progressable,
-      ImmutableClassesGiraphConfiguration<?, ?, ?, ?> configuration,
-      CentralizedServiceWorker<?, ?, ?, ?> serviceWorker) {
+      ImmutableClassesGiraphConfiguration<?, ?, ?> configuration,
+      CentralizedServiceWorker<?, ?, ?> serviceWorker) {
     this.serviceWorker = serviceWorker;
     this.workerClient = serviceWorker.getWorkerClient();
     this.progressable = progressable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
index 9c09524..28f3656 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
@@ -50,20 +50,19 @@ import java.util.Map;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class NettyWorkerClient<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    WorkerClient<I, V, E, M>, ResetSuperstepMetricsObserver {
+    V extends Writable, E extends Writable> implements
+    WorkerClient<I, V, E>, ResetSuperstepMetricsObserver {
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(NettyWorkerClient.class);
   /** Hadoop configuration */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Netty client that does that actual I/O */
   private final NettyClient nettyClient;
   /** Centralized service, needed to get vertex ranges */
-  private final CentralizedServiceWorker<I, V, E, M> service;
+  private final CentralizedServiceWorker<I, V, E> service;
 
   // Metrics
   /** Per-superstep, per-request counters */
@@ -78,8 +77,8 @@ public class NettyWorkerClient<I extends WritableComparable,
    */
   public NettyWorkerClient(
       Mapper<?, ?, ?, ?>.Context context,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
-      CentralizedServiceWorker<I, V, E, M> service) {
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
+      CentralizedServiceWorker<I, V, E> service) {
     this.nettyClient =
         new NettyClient(context, configuration, service.getWorkerInfo());
     this.conf = configuration;
@@ -111,7 +110,7 @@ public class NettyWorkerClient<I extends WritableComparable,
         metrics.getCounter(MetricNames.SEND_AGGREGATORS_TO_WORKER_REQUESTS));
   }
 
-  public CentralizedServiceWorker<I, V, E, M> getService() {
+  public CentralizedServiceWorker<I, V, E> getService() {
     return service;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
index db4ff5d..06965ca 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
@@ -71,26 +71,25 @@ import static org.apache.giraph.conf.GiraphConstants.MAX_MUTATIONS_PER_REQUEST;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("unchecked")
 public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    WorkerClientRequestProcessor<I, V, E, M> {
+    V extends Writable, E extends Writable> implements
+    WorkerClientRequestProcessor<I, V, E> {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(NettyWorkerClientRequestProcessor.class);
   /** Cached partitions of vertices to send */
-  private final SendPartitionCache<I, V, E, M> sendPartitionCache;
+  private final SendPartitionCache<I, V, E> sendPartitionCache;
   /** Cached map of partitions to vertex indices to messages */
-  private final SendMessageCache<I, M> sendMessageCache;
+  private final SendMessageCache<I, Writable> sendMessageCache;
   /** Cache of edges to be sent. */
   private final SendEdgeCache<I, E> sendEdgeCache;
   /** Cached map of partitions to vertex indices to mutations */
-  private final SendMutationsCache<I, V, E, M> sendMutationsCache =
-      new SendMutationsCache<I, V, E, M>();
+  private final SendMutationsCache<I, V, E> sendMutationsCache =
+      new SendMutationsCache<I, V, E>();
   /** NettyClient that could be shared among one or more instances */
-  private final WorkerClient<I, V, E, M> workerClient;
+  private final WorkerClient<I, V, E> workerClient;
   /** Messages sent during the last superstep */
   private long totalMsgsSentInSuperstep = 0;
   /** Maximum size of messages per remote worker to cache before sending */
@@ -100,11 +99,11 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
   /** Maximum number of mutations per partition before sending */
   private final int maxMutationsPerPartition;
   /** Giraph configuration */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /** Service worker */
-  private final CentralizedServiceWorker<I, V, E, M> serviceWorker;
+  private final CentralizedServiceWorker<I, V, E> serviceWorker;
   /** Server data from the server (used for local requests) */
-  private final ServerData<I, V, E, M> serverData;
+  private final ServerData<I, V, E> serverData;
 
   // Per-Superstep Metrics
   /** Number of requests that went on the wire */
@@ -121,13 +120,13 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
    */
   public NettyWorkerClientRequestProcessor(
       Mapper<?, ?, ?, ?>.Context context,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf,
-      CentralizedServiceWorker<I, V, E, M> serviceWorker) {
+      ImmutableClassesGiraphConfiguration<I, V, E> conf,
+      CentralizedServiceWorker<I, V, E> serviceWorker) {
     this.workerClient = serviceWorker.getWorkerClient();
     this.configuration = conf;
 
-    sendPartitionCache = new SendPartitionCache<I, V, E, M>(context, conf);
-    sendMessageCache = new SendMessageCache<I, M>(conf, serviceWorker);
+    sendPartitionCache = new SendPartitionCache<I, V, E>(context, conf);
+    sendMessageCache = new SendMessageCache<I, Writable>(conf, serviceWorker);
     sendEdgeCache = new SendEdgeCache<I, E>(conf, serviceWorker);
     maxMessagesSizePerWorker = MAX_MSG_REQUEST_SIZE.get(conf);
     maxEdgesSizePerWorker = MAX_EDGE_REQUEST_SIZE.get(conf);
@@ -160,7 +159,7 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
   }
 
   @Override
-  public boolean sendMessageRequest(I destVertexId, M message) {
+  public boolean sendMessageRequest(I destVertexId, Writable message) {
     PartitionOwner owner =
         serviceWorker.getVertexPartitionOwner(destVertexId);
     WorkerInfo workerInfo = owner.getWorkerInfo();
@@ -178,11 +177,11 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
     // Send a request if the cache of outgoing message to
     // the remote worker 'workerInfo' is full enough to be flushed
     if (workerMessageSize >= maxMessagesSizePerWorker) {
-      PairList<Integer, ByteArrayVertexIdMessages<I, M>>
+      PairList<Integer, ByteArrayVertexIdMessages<I, Writable>>
           workerMessages =
           sendMessageCache.removeWorkerMessages(workerInfo);
       WritableRequest writableRequest =
-          new SendWorkerMessagesRequest<I, M>(workerMessages);
+          new SendWorkerMessagesRequest<I, Writable>(workerMessages);
       doRequest(workerInfo, writableRequest);
       return true;
     }
@@ -192,14 +191,13 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
 
   @Override
   public void sendPartitionRequest(WorkerInfo workerInfo,
-                                   Partition<I, V, E, M> partition) {
+                                   Partition<I, V, E> partition) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("sendVertexRequest: Sending to " + workerInfo +
           ", with partition " + partition);
     }
 
-    WritableRequest vertexRequest =
-        new SendVertexRequest<I, V, E, M>(partition);
+    WritableRequest vertexRequest = new SendVertexRequest<I, V, E>(partition);
     doRequest(workerInfo, vertexRequest);
 
     // Messages are stored separately
@@ -215,12 +213,13 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
    * @param partition Partition whose messages to send
    */
   private void sendPartitionMessages(WorkerInfo workerInfo,
-                                     Partition<I, V, E, M> partition) {
+                                     Partition<I, V, E> partition) {
     final int partitionId = partition.getId();
-    MessageStoreByPartition<I, M> messageStore =
+    MessageStoreByPartition<I, Writable> messageStore =
         serverData.getCurrentMessageStore();
-    ByteArrayVertexIdMessages<I, M> vertexIdMessages =
-        new ByteArrayVertexIdMessages<I, M>();
+    ByteArrayVertexIdMessages<I, Writable> vertexIdMessages =
+        new ByteArrayVertexIdMessages<I, Writable>(
+            configuration.getOutgoingMessageValueClass());
     vertexIdMessages.setConf(configuration);
     vertexIdMessages.initialize();
     for (I vertexId :
@@ -228,8 +227,8 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
       try {
         // Messages cannot be re-used from this iterable, but add()
         // serializes the message, making this safe
-        Iterable<M> messages = messageStore.getVertexMessages(vertexId);
-        for (M message : messages) {
+        Iterable<Writable> messages = messageStore.getVertexMessages(vertexId);
+        for (Writable message : messages) {
           vertexIdMessages.add(vertexId, message);
         }
       } catch (IOException e) {
@@ -238,18 +237,19 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
       }
       if (vertexIdMessages.getSize() > maxMessagesSizePerWorker) {
         WritableRequest messagesRequest = new
-            SendPartitionCurrentMessagesRequest<I, V, E, M>(
+            SendPartitionCurrentMessagesRequest<I, V, E, Writable>(
             partitionId, vertexIdMessages);
         doRequest(workerInfo, messagesRequest);
         vertexIdMessages =
-            new ByteArrayVertexIdMessages<I, M>();
+            new ByteArrayVertexIdMessages<I, Writable>(
+                configuration.getOutgoingMessageValueClass());
         vertexIdMessages.setConf(configuration);
         vertexIdMessages.initialize();
       }
     }
     if (!vertexIdMessages.isEmpty()) {
       WritableRequest messagesRequest = new
-          SendPartitionCurrentMessagesRequest<I, V, E, M>(
+          SendPartitionCurrentMessagesRequest<I, V, E, Writable>(
           partitionId, vertexIdMessages);
       doRequest(workerInfo, messagesRequest);
     }
@@ -257,8 +257,8 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
 
   @Override
   public void sendVertexRequest(PartitionOwner partitionOwner,
-                                Vertex<I, V, E, M> vertex) {
-    Partition<I, V, E, M> partition =
+                                Vertex<I, V, E> vertex) {
+    Partition<I, V, E> partition =
         sendPartitionCache.addVertex(partitionOwner, vertex);
     if (partition == null) {
       return;
@@ -329,10 +329,10 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
       int partitionMutationCount) {
     // Send a request if enough mutations are there for a partition
     if (partitionMutationCount >= maxMutationsPerPartition) {
-      Map<I, VertexMutations<I, V, E, M>> partitionMutations =
+      Map<I, VertexMutations<I, V, E>> partitionMutations =
           sendMutationsCache.removePartitionMutations(partitionId);
       WritableRequest writableRequest =
-          new SendPartitionMutationsRequest<I, V, E, M>(
+          new SendPartitionMutationsRequest<I, V, E>(
               partitionId, partitionMutations);
       doRequest(partitionOwner.getWorkerInfo(), writableRequest);
     }
@@ -360,7 +360,7 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
   }
 
   @Override
-  public void addVertexRequest(Vertex<I, V, E, M> vertex) throws IOException {
+  public void addVertexRequest(Vertex<I, V, E> vertex) throws IOException {
     PartitionOwner partitionOwner =
         serviceWorker.getVertexPartitionOwner(vertex.getId());
     int partitionId = partitionOwner.getPartitionId();
@@ -398,7 +398,7 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
   @Override
   public void flush() throws IOException {
     // Execute the remaining send partitions (if any)
-    for (Map.Entry<PartitionOwner, Partition<I, V, E, M>> entry :
+    for (Map.Entry<PartitionOwner, Partition<I, V, E>> entry :
         sendPartitionCache.getOwnerPartitionMap().entrySet()) {
       sendPartitionRequest(entry.getKey().getWorkerInfo(), entry.getValue());
     }
@@ -406,15 +406,15 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
 
     // Execute the remaining sends messages (if any)
     PairList<WorkerInfo, PairList<Integer,
-        ByteArrayVertexIdMessages<I, M>>>
+        ByteArrayVertexIdMessages<I, Writable>>>
         remainingMessageCache = sendMessageCache.removeAllMessages();
     PairList<WorkerInfo,
-        PairList<Integer, ByteArrayVertexIdMessages<I, M>>>.Iterator
+        PairList<Integer, ByteArrayVertexIdMessages<I, Writable>>>.Iterator
         iterator = remainingMessageCache.getIterator();
     while (iterator.hasNext()) {
       iterator.next();
       WritableRequest writableRequest =
-          new SendWorkerMessagesRequest<I, M>(
+          new SendWorkerMessagesRequest<I, Writable>(
               iterator.getCurrentSecond());
       doRequest(iterator.getCurrentFirst(), writableRequest);
     }
@@ -435,12 +435,12 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
     }
 
     // Execute the remaining sends mutations (if any)
-    Map<Integer, Map<I, VertexMutations<I, V, E, M>>> remainingMutationsCache =
+    Map<Integer, Map<I, VertexMutations<I, V, E>>> remainingMutationsCache =
         sendMutationsCache.removeAllPartitionMutations();
-    for (Map.Entry<Integer, Map<I, VertexMutations<I, V, E, M>>> entry :
+    for (Map.Entry<Integer, Map<I, VertexMutations<I, V, E>>> entry :
         remainingMutationsCache.entrySet()) {
       WritableRequest writableRequest =
-          new SendPartitionMutationsRequest<I, V, E, M>(
+          new SendPartitionMutationsRequest<I, V, E>(
               entry.getKey(), entry.getValue());
       PartitionOwner partitionOwner =
           serviceWorker.getVertexPartitionOwner(

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
index ed0861e..b457038 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
@@ -22,17 +22,15 @@ import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.WorkerServer;
 import org.apache.giraph.comm.messages.BasicMessageStore;
-import org.apache.giraph.comm.messages.ByteArrayMessagesPerVertexStore;
 import org.apache.giraph.comm.messages.DiskBackedMessageStore;
 import org.apache.giraph.comm.messages.DiskBackedMessageStoreByPartition;
 import org.apache.giraph.comm.messages.FlushableMessageStore;
+import org.apache.giraph.comm.messages.InMemoryMessageStoreFactory;
 import org.apache.giraph.comm.messages.MessageStoreByPartition;
 import org.apache.giraph.comm.messages.MessageStoreFactory;
-import org.apache.giraph.comm.messages.OneMessagePerVertexStore;
 import org.apache.giraph.comm.messages.SequentialFileMessageStore;
 import org.apache.giraph.comm.netty.handler.WorkerRequestServerHandler;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexMutations;
 import org.apache.giraph.graph.VertexResolver;
@@ -60,23 +58,24 @@ import static org.apache.giraph.conf.GiraphConstants.USE_OUT_OF_CORE_MESSAGES;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class NettyWorkerServer<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements WorkerServer<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    implements WorkerServer<I, V, E> {
   /** Class logger */
   private static final Logger LOG =
     Logger.getLogger(NettyWorkerServer.class);
   /** Hadoop configuration */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Service worker */
-  private final CentralizedServiceWorker<I, V, E, M> service;
+  private final CentralizedServiceWorker<I, V, E> service;
   /** Netty server that does that actual I/O */
   private final NettyServer nettyServer;
   /** Server data storage */
-  private final ServerData<I, V, E, M> serverData;
+  private final ServerData<I, V, E> serverData;
+  /** Mapper context */
+  private final Mapper<?, ?, ?, ?>.Context context;
 
   /**
    * Constructor to start the server.
@@ -85,18 +84,19 @@ public class NettyWorkerServer<I extends WritableComparable,
    * @param service Service to get partition mappings
    * @param context Mapper context
    */
-  public NettyWorkerServer(ImmutableClassesGiraphConfiguration<I, V, E, M> conf,
-      CentralizedServiceWorker<I, V, E, M> service,
+  public NettyWorkerServer(ImmutableClassesGiraphConfiguration<I, V, E> conf,
+      CentralizedServiceWorker<I, V, E> service,
       Mapper<?, ?, ?, ?>.Context context) {
     this.conf = conf;
     this.service = service;
+    this.context = context;
 
     serverData =
-        new ServerData<I, V, E, M>(service, conf, createMessageStoreFactory(),
+        new ServerData<I, V, E>(service, conf, createMessageStoreFactory(),
             context);
 
     nettyServer = new NettyServer(conf,
-        new WorkerRequestServerHandler.Factory<I, V, E, M>(serverData),
+        new WorkerRequestServerHandler.Factory<I, V, E>(serverData),
         service.getWorkerInfo(), context);
     nettyServer.start();
   }
@@ -107,33 +107,20 @@ public class NettyWorkerServer<I extends WritableComparable,
    *
    * @return Message store factory
    */
-  private MessageStoreFactory<I, M, MessageStoreByPartition<I, M>>
+  private MessageStoreFactory<I, Writable, MessageStoreByPartition<I, Writable>>
   createMessageStoreFactory() {
     boolean useOutOfCoreMessaging = USE_OUT_OF_CORE_MESSAGES.get(conf);
     if (!useOutOfCoreMessaging) {
-      if (conf.useCombiner()) {
-        if (LOG.isInfoEnabled()) {
-          LOG.info("createMessageStoreFactory: " +
-              "Using OneMessagePerVertexStore since combiner enabled");
-        }
-        return OneMessagePerVertexStore.newFactory(service, conf);
-      } else {
-        if (LOG.isInfoEnabled()) {
-          LOG.info("createMessageStoreFactory: " +
-              "Using ByteArrayMessagesPerVertexStore " +
-              "since there is no combiner");
-        }
-        return ByteArrayMessagesPerVertexStore.newFactory(service, conf);
-      }
+      return new InMemoryMessageStoreFactory<I, Writable>(service, conf);
     } else {
       int maxMessagesInMemory = MAX_MESSAGES_IN_MEMORY.get(conf);
       if (LOG.isInfoEnabled()) {
         LOG.info("createMessageStoreFactory: Using DiskBackedMessageStore, " +
             "maxMessagesInMemory = " + maxMessagesInMemory);
       }
-      MessageStoreFactory<I, M, BasicMessageStore<I, M>> fileStoreFactory =
-          SequentialFileMessageStore.newFactory(conf);
-      MessageStoreFactory<I, M, FlushableMessageStore<I, M>>
+      MessageStoreFactory<I, Writable, BasicMessageStore<I, Writable>>
+          fileStoreFactory = SequentialFileMessageStore.newFactory(conf);
+      MessageStoreFactory<I, Writable, FlushableMessageStore<I, Writable>>
           partitionStoreFactory =
           DiskBackedMessageStore.newFactory(conf, fileStoreFactory);
       return DiskBackedMessageStoreByPartition.newFactory(service,
@@ -147,21 +134,19 @@ public class NettyWorkerServer<I extends WritableComparable,
   }
 
   @Override
-  public void prepareSuperstep(GraphState<I, V, E, M> graphState) {
+  public void prepareSuperstep() {
     serverData.prepareSuperstep();
-    resolveMutations(graphState);
+    resolveMutations();
   }
 
   /**
    * Resolve mutation requests.
-   *
-   * @param graphState Graph state
    */
-  private void resolveMutations(GraphState<I, V, E, M> graphState) {
+  private void resolveMutations() {
     Multimap<Integer, I> resolveVertexIndices = HashMultimap.create(
         service.getPartitionStore().getNumPartitions(), 100);
       // Add any mutated vertex indices to be resolved
-    for (Entry<I, VertexMutations<I, V, E, M>> e :
+    for (Entry<I, VertexMutations<I, V, E>> e :
         serverData.getVertexMutations().entrySet()) {
       I vertexId = e.getKey();
       Integer partitionId = service.getPartitionId(vertexId);
@@ -176,7 +161,7 @@ public class NettyWorkerServer<I extends WritableComparable,
       Iterable<I> destinations = serverData.getCurrentMessageStore().
           getPartitionDestinationVertices(partitionId);
       if (!Iterables.isEmpty(destinations)) {
-        Partition<I, V, E, M> partition =
+        Partition<I, V, E> partition =
             service.getPartitionStore().getPartition(partitionId);
         for (I vertexId : destinations) {
           if (partition.getVertex(vertexId) == null) {
@@ -191,18 +176,17 @@ public class NettyWorkerServer<I extends WritableComparable,
       }
     }
     // Resolve all graph mutations
-    VertexResolver<I, V, E, M> vertexResolver =
-        conf.createVertexResolver(graphState);
+    VertexResolver<I, V, E> vertexResolver = conf.createVertexResolver();
     for (Entry<Integer, Collection<I>> e :
         resolveVertexIndices.asMap().entrySet()) {
-      Partition<I, V, E, M> partition =
+      Partition<I, V, E> partition =
           service.getPartitionStore().getPartition(e.getKey());
       for (I vertexIndex : e.getValue()) {
-        Vertex<I, V, E, M> originalVertex =
+        Vertex<I, V, E> originalVertex =
             partition.getVertex(vertexIndex);
 
-        VertexMutations<I, V, E, M> mutations = null;
-        VertexMutations<I, V, E, M> vertexMutations =
+        VertexMutations<I, V, E> mutations = null;
+        VertexMutations<I, V, E> vertexMutations =
             serverData.getVertexMutations().get(vertexIndex);
         if (vertexMutations != null) {
           synchronized (vertexMutations) {
@@ -210,11 +194,11 @@ public class NettyWorkerServer<I extends WritableComparable,
           }
           serverData.getVertexMutations().remove(vertexIndex);
         }
-        Vertex<I, V, E, M> vertex = vertexResolver.resolve(
+        Vertex<I, V, E> vertex = vertexResolver.resolve(
             vertexIndex, originalVertex, mutations,
             serverData.getCurrentMessageStore().
                 hasMessagesForVertex(vertexIndex));
-        graphState.getContext().progress();
+        context.progress();
 
         if (LOG.isDebugEnabled()) {
           LOG.debug("resolveMutations: Resolved vertex index " +
@@ -240,7 +224,7 @@ public class NettyWorkerServer<I extends WritableComparable,
   }
 
   @Override
-  public ServerData<I, V, E, M> getServerData() {
+  public ServerData<I, V, E> getServerData() {
     return serverData;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
index b4e7dda..f64c373 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public class WorkerRequestServerHandler<I extends WritableComparable,
     V extends Writable, E extends Writable, M extends Writable> extends
-    RequestServerHandler<WorkerRequest<I, V, E, M>> {
+    RequestServerHandler<WorkerRequest<I, V, E>> {
   /** Data that can be accessed for handling requests */
-  private final ServerData<I, V, E, M> serverData;
+  private final ServerData<I, V, E> serverData;
 
   /**
    * Constructor with external server data
@@ -47,7 +47,7 @@ public class WorkerRequestServerHandler<I extends WritableComparable,
    * @param conf                     Configuration
    * @param myTaskInfo               Current task info
    */
-  public WorkerRequestServerHandler(ServerData<I, V, E, M> serverData,
+  public WorkerRequestServerHandler(ServerData<I, V, E> serverData,
       WorkerRequestReservedMap workerRequestReservedMap,
       ImmutableClassesGiraphConfiguration conf,
       TaskInfo myTaskInfo) {
@@ -56,23 +56,23 @@ public class WorkerRequestServerHandler<I extends WritableComparable,
   }
 
   @Override
-  public void processRequest(WorkerRequest<I, V, E, M> request) {
+  public void processRequest(WorkerRequest<I, V, E> request) {
     request.doRequest(serverData);
   }
 
   /** Factory for {@link WorkerRequestServerHandler} */
   public static class Factory<I extends WritableComparable,
-      V extends Writable, E extends Writable, M extends Writable> implements
+      V extends Writable, E extends Writable> implements
       RequestServerHandler.Factory {
     /** Data that can be accessed for handling requests */
-    private final ServerData<I, V, E, M> serverData;
+    private final ServerData<I, V, E> serverData;
 
     /**
      * Constructor
      *
      * @param serverData Data held by the server
      */
-    public Factory(ServerData<I, V, E, M> serverData) {
+    public Factory(ServerData<I, V, E> serverData) {
       this.serverData = serverData;
     }
 
@@ -81,8 +81,8 @@ public class WorkerRequestServerHandler<I extends WritableComparable,
         WorkerRequestReservedMap workerRequestReservedMap,
         ImmutableClassesGiraphConfiguration conf,
         TaskInfo myTaskInfo) {
-      return new WorkerRequestServerHandler<I, V, E,
-          M>(serverData, workerRequestReservedMap, conf, myTaskInfo);
+      return new WorkerRequestServerHandler<I, V, E, Writable>(serverData,
+          workerRequestReservedMap, conf, myTaskInfo);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionCurrentMessagesRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionCurrentMessagesRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionCurrentMessagesRequest.java
index 037f4a0..88641c5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionCurrentMessagesRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionCurrentMessagesRequest.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public class SendPartitionCurrentMessagesRequest<I extends WritableComparable,
   V extends Writable, E extends Writable, M extends Writable> extends
-  WritableRequest<I, V, E, M> implements WorkerRequest<I, V, E, M> {
+  WritableRequest<I, V, E> implements WorkerRequest<I, V, E> {
   /** Destination partition for these vertices' messages*/
   private int partitionId;
   /** Map of destination vertex ID's to message lists */
@@ -67,7 +67,10 @@ public class SendPartitionCurrentMessagesRequest<I extends WritableComparable,
   @Override
   public void readFieldsRequest(DataInput input) throws IOException {
     partitionId = input.readInt();
-    vertexIdMessageMap = new ByteArrayVertexIdMessages<I, M>();
+    // At this moment the Computation class have already been replaced with
+    // the new one, and we deal with messages from previous superstep
+    vertexIdMessageMap = new ByteArrayVertexIdMessages<I, M>(
+        getConf().getIncomingMessageValueClass());
     vertexIdMessageMap.setConf(getConf());
     vertexIdMessageMap.initialize();
     vertexIdMessageMap.readFields(input);
@@ -80,9 +83,9 @@ public class SendPartitionCurrentMessagesRequest<I extends WritableComparable,
   }
 
   @Override
-  public void doRequest(ServerData<I, V, E, M> serverData) {
+  public void doRequest(ServerData<I, V, E> serverData) {
     try {
-      serverData.getCurrentMessageStore().addPartitionMessages(partitionId,
+      serverData.<M>getCurrentMessageStore().addPartitionMessages(partitionId,
           vertexIdMessageMap);
     } catch (IOException e) {
       throw new RuntimeException("doRequest: Got IOException ", e);

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionMutationsRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionMutationsRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionMutationsRequest.java
index a96842d..de0d098 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionMutationsRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendPartitionMutationsRequest.java
@@ -42,19 +42,18 @@ import java.util.concurrent.ConcurrentHashMap;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class SendPartitionMutationsRequest<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> extends
-    WritableRequest<I, V, E, M> implements WorkerRequest<I, V, E, M> {
+    V extends Writable, E extends Writable> extends
+    WritableRequest<I, V, E> implements WorkerRequest<I, V, E> {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(SendPartitionMutationsRequest.class);
   /** Partition id */
   private int partitionId;
   /** Mutations sent for a partition */
-  private Map<I, VertexMutations<I, V, E, M>> vertexIdMutations;
+  private Map<I, VertexMutations<I, V, E>> vertexIdMutations;
 
   /**
    * Constructor used for reflection only
@@ -69,7 +68,7 @@ public class SendPartitionMutationsRequest<I extends WritableComparable,
    */
   public SendPartitionMutationsRequest(
       int partitionId,
-      Map<I, VertexMutations<I, V, E, M>> vertexIdMutations) {
+      Map<I, VertexMutations<I, V, E>> vertexIdMutations) {
     this.partitionId = partitionId;
     this.vertexIdMutations = vertexIdMutations;
   }
@@ -82,8 +81,8 @@ public class SendPartitionMutationsRequest<I extends WritableComparable,
     for (int i = 0; i < vertexIdMutationsSize; ++i) {
       I vertexId = getConf().createVertexId();
       vertexId.readFields(input);
-      VertexMutations<I, V, E, M> vertexMutations =
-          new VertexMutations<I, V, E, M>();
+      VertexMutations<I, V, E> vertexMutations =
+          new VertexMutations<I, V, E>();
       vertexMutations.setConf(getConf());
       vertexMutations.readFields(input);
       if (vertexIdMutations.put(vertexId, vertexMutations) != null) {
@@ -97,7 +96,7 @@ public class SendPartitionMutationsRequest<I extends WritableComparable,
   public void writeRequest(DataOutput output) throws IOException {
     output.writeInt(partitionId);
     output.writeInt(vertexIdMutations.size());
-    for (Entry<I, VertexMutations<I, V, E, M>> entry :
+    for (Entry<I, VertexMutations<I, V, E>> entry :
         vertexIdMutations.entrySet()) {
       entry.getKey().write(output);
       entry.getValue().write(output);
@@ -110,15 +109,15 @@ public class SendPartitionMutationsRequest<I extends WritableComparable,
   }
 
   @Override
-  public void doRequest(ServerData<I, V, E, M> serverData) {
-    ConcurrentHashMap<I, VertexMutations<I, V, E, M>> vertexMutations =
+  public void doRequest(ServerData<I, V, E> serverData) {
+    ConcurrentHashMap<I, VertexMutations<I, V, E>> vertexMutations =
       serverData.getVertexMutations();
     Histogram verticesInMutationHist = GiraphMetrics.get().perSuperstep()
         .getUniformHistogram(MetricNames.VERTICES_IN_MUTATION_REQUEST);
     verticesInMutationHist.update(vertexMutations.size());
-    for (Entry<I, VertexMutations<I, V, E, M>> entry :
+    for (Entry<I, VertexMutations<I, V, E>> entry :
         vertexIdMutations.entrySet()) {
-      VertexMutations<I, V, E, M> mutations =
+      VertexMutations<I, V, E> mutations =
           vertexMutations.get(entry.getKey());
       if (mutations == null) {
         mutations = vertexMutations.putIfAbsent(

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
index 1de3cbb..e0cb916 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendVertexRequest.java
@@ -34,17 +34,16 @@ import java.io.IOException;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class SendVertexRequest<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> extends
-    WritableRequest<I, V, E, M> implements WorkerRequest<I, V, E, M> {
+    V extends Writable, E extends Writable> extends
+    WritableRequest<I, V, E> implements WorkerRequest<I, V, E> {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(SendVertexRequest.class);
   /** Partition */
-  private Partition<I, V, E, M> partition;
+  private Partition<I, V, E> partition;
 
   /**
    * Constructor used for reflection only
@@ -56,7 +55,7 @@ public class SendVertexRequest<I extends WritableComparable,
    *
    * @param partition Partition to send the request to
    */
-  public SendVertexRequest(Partition<I, V, E, M> partition) {
+  public SendVertexRequest(Partition<I, V, E> partition) {
     this.partition = partition;
   }
 
@@ -77,7 +76,7 @@ public class SendVertexRequest<I extends WritableComparable,
   }
 
   @Override
-  public void doRequest(ServerData<I, V, E, M> serverData) {
+  public void doRequest(ServerData<I, V, E> serverData) {
     serverData.getPartitionStore().addPartition(partition);
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerMessagesRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerMessagesRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerMessagesRequest.java
index 04b633b..f6bf9bf 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerMessagesRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/SendWorkerMessagesRequest.java
@@ -54,7 +54,8 @@ public class SendWorkerMessagesRequest<I extends WritableComparable,
 
   @Override
   public ByteArrayVertexIdMessages<I, M> createByteArrayVertexIdData() {
-    return new ByteArrayVertexIdMessages<I, M>();
+    return new ByteArrayVertexIdMessages<I, M>(
+        getConf().getOutgoingMessageValueClass());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/requests/WorkerRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/WorkerRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/WorkerRequest.java
index 4d9382f..0ceb3eb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/WorkerRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/WorkerRequest.java
@@ -28,14 +28,13 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public interface WorkerRequest<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /**
    * Execute the request
    *
    * @param serverData Accessible data that can be mutated per the request
    */
-  void doRequest(ServerData<I, V, E, M> serverData);
+  void doRequest(ServerData<I, V, E> serverData);
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java b/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
index fad20b0..181e681 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
@@ -32,12 +32,10 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public abstract class WritableRequest<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements Writable,
-    ImmutableClassesGiraphConfigurable<I, V, E, M> {
+    V extends Writable, E extends Writable> implements Writable,
+    ImmutableClassesGiraphConfigurable<I, V, E> {
   /**
    * Value to use when size of the request in serialized form is not known
    * or too expensive to calculate
@@ -45,7 +43,7 @@ public abstract class WritableRequest<I extends WritableComparable,
   public static final int UNKNOWN_SIZE = -1;
 
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
   /** Client id */
   private int clientId = -1;
   /** Request id */
@@ -103,13 +101,12 @@ public abstract class WritableRequest<I extends WritableComparable,
   abstract void writeRequest(DataOutput output) throws IOException;
 
   @Override
-  public final ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
+  public final ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
     return conf;
   }
 
   @Override
-  public final void setConf(ImmutableClassesGiraphConfiguration<I, V,
-      E, M> conf) {
+  public final void setConf(ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     this.conf = conf;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/conf/AllOptions.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/AllOptions.java b/giraph-core/src/main/java/org/apache/giraph/conf/AllOptions.java
index cceaaef..5d150d0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/AllOptions.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/AllOptions.java
@@ -24,7 +24,7 @@ import com.google.common.collect.Lists;
 import java.util.Collections;
 import java.util.List;
 
-import static org.apache.giraph.conf.GiraphConstants.VERTEX_CLASS;
+import static org.apache.giraph.conf.GiraphConstants.COMPUTATION_CLASS;
 
 /**
  * Tracks all of the Giraph options
@@ -75,7 +75,7 @@ public class AllOptions {
   public static void main(String[] args) {
     // This is necessary to trigger the static constants in GiraphConstants to
     // get loaded. Without it we get no output.
-    VERTEX_CLASS.toString();
+    COMPUTATION_CLASS.toString();
 
     LOG.info(allOptionsString());
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java b/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java
index 77564ee..c5096fa 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java
@@ -27,22 +27,21 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public class DefaultImmutableClassesGiraphConfigurable<
     I extends WritableComparable, V extends Writable,
-    E extends Writable, M extends Writable> implements
-    ImmutableClassesGiraphConfigurable<I, V, E, M> {
+    E extends Writable> implements
+    ImmutableClassesGiraphConfigurable<I, V, E> {
   /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+  private ImmutableClassesGiraphConfiguration<I, V, E> conf;
 
   @Override
-  public void setConf(ImmutableClassesGiraphConfiguration<I, V, E, M> conf) {
+  public void setConf(ImmutableClassesGiraphConfiguration<I, V, E> conf) {
     this.conf = conf;
   }
 
   @Override
-  public ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
+  public ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
     return conf;
   }
 }


[04/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpoint.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpoint.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpoint.java
new file mode 100644
index 0000000..984e079
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpoint.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.PosixParser;
+import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
+import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
+import org.apache.giraph.job.GiraphJob;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * An example that simply uses its id, value, and edges to compute new data
+ * every iteration to verify that checkpoint restarting works.  Fault injection
+ * can also test automated checkpoint restarts.
+ */
+public class SimpleCheckpoint implements Tool {
+  /** Which superstep to cause the worker to fail */
+  public static final int FAULTING_SUPERSTEP = 4;
+  /** Vertex id to fault on */
+  public static final long FAULTING_VERTEX_ID = 1;
+  /** Dynamically set number of supersteps */
+  public static final String SUPERSTEP_COUNT =
+      "simpleCheckpointVertex.superstepCount";
+  /** Should fault? */
+  public static final String ENABLE_FAULT =
+      "simpleCheckpointVertex.enableFault";
+  /** Class logger */
+  private static final Logger LOG =
+      Logger.getLogger(SimpleCheckpoint.class);
+  /** Configuration */
+  private Configuration conf;
+
+  /**
+   * Actual computation.
+   */
+  public static class SimpleCheckpointComputation extends
+      BasicComputation<LongWritable, IntWritable, FloatWritable,
+          FloatWritable> {
+    @Override
+    public void compute(
+        Vertex<LongWritable, IntWritable, FloatWritable> vertex,
+        Iterable<FloatWritable> messages) throws IOException {
+      SimpleCheckpointVertexWorkerContext workerContext = getWorkerContext();
+
+      boolean enableFault = workerContext.getEnableFault();
+      int supersteps = workerContext.getSupersteps();
+
+      if (enableFault && (getSuperstep() == FAULTING_SUPERSTEP) &&
+          (getContext().getTaskAttemptID().getId() == 0) &&
+          (vertex.getId().get() == FAULTING_VERTEX_ID)) {
+        LOG.info("compute: Forced a fault on the first " +
+            "attempt of superstep " +
+            FAULTING_SUPERSTEP + " and vertex id " +
+            FAULTING_VERTEX_ID);
+        System.exit(-1);
+      }
+      if (getSuperstep() > supersteps) {
+        vertex.voteToHalt();
+        return;
+      }
+      long sumAgg = this.<LongWritable>getAggregatedValue(
+          LongSumAggregator.class.getName()).get();
+      LOG.info("compute: " + sumAgg);
+      aggregate(LongSumAggregator.class.getName(),
+          new LongWritable(vertex.getId().get()));
+      LOG.info("compute: sum = " + sumAgg +
+          " for vertex " + vertex.getId());
+      float msgValue = 0.0f;
+      for (FloatWritable message : messages) {
+        float curMsgValue = message.get();
+        msgValue += curMsgValue;
+        LOG.info("compute: got msgValue = " + curMsgValue +
+            " for vertex " + vertex.getId() +
+            " on superstep " + getSuperstep());
+      }
+      int vertexValue = vertex.getValue().get();
+      vertex.setValue(new IntWritable(vertexValue + (int) msgValue));
+      LOG.info("compute: vertex " + vertex.getId() +
+          " has value " + vertex.getValue() +
+          " on superstep " + getSuperstep());
+      for (Edge<LongWritable, FloatWritable> edge : vertex.getEdges()) {
+        FloatWritable newEdgeValue = new FloatWritable(edge.getValue().get() +
+            (float) vertexValue);
+        Edge<LongWritable, FloatWritable> newEdge =
+            EdgeFactory.create(edge.getTargetVertexId(), newEdgeValue);
+        LOG.info("compute: vertex " + vertex.getId() +
+            " sending edgeValue " + edge.getValue() +
+            " vertexValue " + vertexValue +
+            " total " + newEdgeValue +
+            " to vertex " + edge.getTargetVertexId() +
+            " on superstep " + getSuperstep());
+        vertex.addEdge(newEdge);
+        sendMessage(edge.getTargetVertexId(), newEdgeValue);
+      }
+    }
+  }
+
+  /**
+   * Worker context associated with {@link SimpleCheckpoint}.
+   */
+  public static class SimpleCheckpointVertexWorkerContext
+      extends WorkerContext {
+    /** Filename to indicate whether a fault was found */
+    public static final String FAULT_FILE = "/tmp/faultFile";
+    /** User can access this after the application finishes if local */
+    private static long FINAL_SUM;
+    /** Number of supersteps to run (6 by default) */
+    private int supersteps = 6;
+    /** Enable the fault at the particular vertex id and superstep? */
+    private boolean enableFault = false;
+
+    public static long getFinalSum() {
+      return FINAL_SUM;
+    }
+
+    @Override
+    public void preApplication()
+      throws InstantiationException, IllegalAccessException {
+      supersteps = getContext().getConfiguration()
+          .getInt(SUPERSTEP_COUNT, supersteps);
+      enableFault = getContext().getConfiguration()
+          .getBoolean(ENABLE_FAULT, false);
+    }
+
+    @Override
+    public void postApplication() {
+      FINAL_SUM = this.<LongWritable>getAggregatedValue(
+          LongSumAggregator.class.getName()).get();
+      LOG.info("FINAL_SUM=" + FINAL_SUM);
+    }
+
+    @Override
+    public void preSuperstep() {
+    }
+
+    @Override
+    public void postSuperstep() { }
+
+    public int getSupersteps() {
+      return this.supersteps;
+    }
+
+    public boolean getEnableFault() {
+      return this.enableFault;
+    }
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    Options options = new Options();
+    options.addOption("h", "help", false, "Help");
+    options.addOption("v", "verbose", false, "Verbose");
+    options.addOption("w",
+        "workers",
+        true,
+        "Number of workers");
+    options.addOption("s",
+        "supersteps",
+        true,
+        "Supersteps to execute before finishing");
+    options.addOption("w",
+        "workers",
+        true,
+        "Minimum number of workers");
+    options.addOption("o",
+        "outputDirectory",
+        true,
+        "Output directory");
+    HelpFormatter formatter = new HelpFormatter();
+    if (args.length == 0) {
+      formatter.printHelp(getClass().getName(), options, true);
+      return 0;
+    }
+    CommandLineParser parser = new PosixParser();
+    CommandLine cmd = parser.parse(options, args);
+    if (cmd.hasOption('h')) {
+      formatter.printHelp(getClass().getName(), options, true);
+      return 0;
+    }
+    if (!cmd.hasOption('w')) {
+      LOG.info("Need to choose the number of workers (-w)");
+      return -1;
+    }
+    if (!cmd.hasOption('o')) {
+      LOG.info("Need to set the output directory (-o)");
+      return -1;
+    }
+
+    GiraphJob bspJob = new GiraphJob(getConf(), getClass().getName());
+    bspJob.getConfiguration().setComputationClass(
+        SimpleCheckpointComputation.class);
+    bspJob.getConfiguration().setVertexInputFormatClass(
+        GeneratedVertexInputFormat.class);
+    bspJob.getConfiguration().setVertexOutputFormatClass(
+        IdWithValueTextOutputFormat.class);
+    bspJob.getConfiguration().setWorkerContextClass(
+        SimpleCheckpointVertexWorkerContext.class);
+    bspJob.getConfiguration().setMasterComputeClass(
+        SimpleCheckpointVertexMasterCompute.class);
+    int minWorkers = Integer.parseInt(cmd.getOptionValue('w'));
+    int maxWorkers = Integer.parseInt(cmd.getOptionValue('w'));
+    bspJob.getConfiguration().setWorkerConfiguration(
+        minWorkers, maxWorkers, 100.0f);
+
+    FileOutputFormat.setOutputPath(bspJob.getInternalJob(),
+                                   new Path(cmd.getOptionValue('o')));
+    boolean verbose = false;
+    if (cmd.hasOption('v')) {
+      verbose = true;
+    }
+    if (cmd.hasOption('s')) {
+      getConf().setInt(SUPERSTEP_COUNT,
+          Integer.parseInt(cmd.getOptionValue('s')));
+    }
+    if (bspJob.run(verbose)) {
+      return 0;
+    } else {
+      return -1;
+    }
+  }
+
+  /**
+   * Master compute associated with {@link SimpleCheckpoint}.
+   * It registers required aggregators.
+   */
+  public static class SimpleCheckpointVertexMasterCompute extends
+      DefaultMasterCompute {
+    @Override
+    public void initialize() throws InstantiationException,
+        IllegalAccessException {
+      registerAggregator(LongSumAggregator.class.getName(),
+          LongSumAggregator.class);
+    }
+  }
+
+  /**
+   * Executable from the command line.
+   *
+   * @param args Command line args.
+   * @throws Exception
+   */
+  public static void main(String[] args) throws Exception {
+    System.exit(ToolRunner.run(new SimpleCheckpoint(), args));
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
deleted file mode 100644
index 03d977b..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
+++ /dev/null
@@ -1,286 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.PosixParser;
-import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
-import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
-import org.apache.giraph.job.GiraphJob;
-import org.apache.giraph.master.DefaultMasterCompute;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.worker.WorkerContext;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.Logger;
-
-/**
- * An example that simply uses its id, value, and edges to compute new data
- * every iteration to verify that checkpoint restarting works.  Fault injection
- * can also test automated checkpoint restarts.
- */
-public class SimpleCheckpointVertex implements Tool {
-  /** Which superstep to cause the worker to fail */
-  public static final int FAULTING_SUPERSTEP = 4;
-  /** Vertex id to fault on */
-  public static final long FAULTING_VERTEX_ID = 1;
-  /** Dynamically set number of supersteps */
-  public static final String SUPERSTEP_COUNT =
-      "simpleCheckpointVertex.superstepCount";
-  /** Should fault? */
-  public static final String ENABLE_FAULT =
-      "simpleCheckpointVertex.enableFault";
-  /** Class logger */
-  private static final Logger LOG =
-      Logger.getLogger(SimpleCheckpointVertex.class);
-  /** Configuration */
-  private Configuration conf;
-
-  /**
-   * Actual computation.
-   */
-  public static class SimpleCheckpointComputation extends
-      Vertex<LongWritable, IntWritable, FloatWritable, FloatWritable> {
-    @Override
-    public void compute(Iterable<FloatWritable> messages) {
-      SimpleCheckpointVertexWorkerContext workerContext =
-          (SimpleCheckpointVertexWorkerContext) getWorkerContext();
-
-      boolean enableFault = workerContext.getEnableFault();
-      int supersteps = workerContext.getSupersteps();
-
-      if (enableFault && (getSuperstep() == FAULTING_SUPERSTEP) &&
-          (getContext().getTaskAttemptID().getId() == 0) &&
-          (getId().get() == FAULTING_VERTEX_ID)) {
-        LOG.info("compute: Forced a fault on the first " +
-            "attempt of superstep " +
-            FAULTING_SUPERSTEP + " and vertex id " +
-            FAULTING_VERTEX_ID);
-        System.exit(-1);
-      }
-      if (getSuperstep() > supersteps) {
-        voteToHalt();
-        return;
-      }
-      long sumAgg = this.<LongWritable>getAggregatedValue(
-          LongSumAggregator.class.getName()).get();
-      LOG.info("compute: " + sumAgg);
-      aggregate(LongSumAggregator.class.getName(),
-          new LongWritable(getId().get()));
-      LOG.info("compute: sum = " + sumAgg +
-          " for vertex " + getId());
-      float msgValue = 0.0f;
-      for (FloatWritable message : messages) {
-        float curMsgValue = message.get();
-        msgValue += curMsgValue;
-        LOG.info("compute: got msgValue = " + curMsgValue +
-            " for vertex " + getId() +
-            " on superstep " + getSuperstep());
-      }
-      int vertexValue = getValue().get();
-      setValue(new IntWritable(vertexValue + (int) msgValue));
-      LOG.info("compute: vertex " + getId() +
-          " has value " + getValue() +
-          " on superstep " + getSuperstep());
-      for (Edge<LongWritable, FloatWritable> edge : getEdges()) {
-        FloatWritable newEdgeValue = new FloatWritable(edge.getValue().get() +
-            (float) vertexValue);
-        Edge<LongWritable, FloatWritable> newEdge =
-            EdgeFactory.create(edge.getTargetVertexId(), newEdgeValue);
-        LOG.info("compute: vertex " + getId() +
-            " sending edgeValue " + edge.getValue() +
-            " vertexValue " + vertexValue +
-            " total " + newEdgeValue +
-            " to vertex " + edge.getTargetVertexId() +
-            " on superstep " + getSuperstep());
-        addEdge(newEdge);
-        sendMessage(edge.getTargetVertexId(), newEdgeValue);
-      }
-    }
-  }
-
-  /**
-   * Worker context associated with {@link SimpleCheckpointVertex}.
-   */
-  public static class SimpleCheckpointVertexWorkerContext
-      extends WorkerContext {
-    /** Filename to indicate whether a fault was found */
-    public static final String FAULT_FILE = "/tmp/faultFile";
-    /** User can access this after the application finishes if local */
-    private static long FINAL_SUM;
-    /** Number of supersteps to run (6 by default) */
-    private int supersteps = 6;
-    /** Enable the fault at the particular vertex id and superstep? */
-    private boolean enableFault = false;
-
-    public static long getFinalSum() {
-      return FINAL_SUM;
-    }
-
-    @Override
-    public void preApplication()
-      throws InstantiationException, IllegalAccessException {
-      supersteps = getContext().getConfiguration()
-          .getInt(SUPERSTEP_COUNT, supersteps);
-      enableFault = getContext().getConfiguration()
-          .getBoolean(ENABLE_FAULT, false);
-    }
-
-    @Override
-    public void postApplication() {
-      FINAL_SUM = this.<LongWritable>getAggregatedValue(
-          LongSumAggregator.class.getName()).get();
-      LOG.info("FINAL_SUM=" + FINAL_SUM);
-    }
-
-    @Override
-    public void preSuperstep() {
-    }
-
-    @Override
-    public void postSuperstep() { }
-
-    public int getSupersteps() {
-      return this.supersteps;
-    }
-
-    public boolean getEnableFault() {
-      return this.enableFault;
-    }
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    Options options = new Options();
-    options.addOption("h", "help", false, "Help");
-    options.addOption("v", "verbose", false, "Verbose");
-    options.addOption("w",
-        "workers",
-        true,
-        "Number of workers");
-    options.addOption("s",
-        "supersteps",
-        true,
-        "Supersteps to execute before finishing");
-    options.addOption("w",
-        "workers",
-        true,
-        "Minimum number of workers");
-    options.addOption("o",
-        "outputDirectory",
-        true,
-        "Output directory");
-    HelpFormatter formatter = new HelpFormatter();
-    if (args.length == 0) {
-      formatter.printHelp(getClass().getName(), options, true);
-      return 0;
-    }
-    CommandLineParser parser = new PosixParser();
-    CommandLine cmd = parser.parse(options, args);
-    if (cmd.hasOption('h')) {
-      formatter.printHelp(getClass().getName(), options, true);
-      return 0;
-    }
-    if (!cmd.hasOption('w')) {
-      LOG.info("Need to choose the number of workers (-w)");
-      return -1;
-    }
-    if (!cmd.hasOption('o')) {
-      LOG.info("Need to set the output directory (-o)");
-      return -1;
-    }
-
-    GiraphJob bspJob = new GiraphJob(getConf(), getClass().getName());
-    bspJob.getConfiguration().setVertexClass(SimpleCheckpointComputation.class);
-    bspJob.getConfiguration().setVertexInputFormatClass(
-        GeneratedVertexInputFormat.class);
-    bspJob.getConfiguration().setVertexOutputFormatClass(
-        IdWithValueTextOutputFormat.class);
-    bspJob.getConfiguration().setWorkerContextClass(
-        SimpleCheckpointVertexWorkerContext.class);
-    bspJob.getConfiguration().setMasterComputeClass(
-        SimpleCheckpointVertexMasterCompute.class);
-    int minWorkers = Integer.parseInt(cmd.getOptionValue('w'));
-    int maxWorkers = Integer.parseInt(cmd.getOptionValue('w'));
-    bspJob.getConfiguration().setWorkerConfiguration(
-        minWorkers, maxWorkers, 100.0f);
-
-    FileOutputFormat.setOutputPath(bspJob.getInternalJob(),
-                                   new Path(cmd.getOptionValue('o')));
-    boolean verbose = false;
-    if (cmd.hasOption('v')) {
-      verbose = true;
-    }
-    if (cmd.hasOption('s')) {
-      getConf().setInt(SUPERSTEP_COUNT,
-          Integer.parseInt(cmd.getOptionValue('s')));
-    }
-    if (bspJob.run(verbose)) {
-      return 0;
-    } else {
-      return -1;
-    }
-  }
-
-  /**
-   * Master compute associated with {@link SimpleCheckpointVertex}.
-   * It registers required aggregators.
-   */
-  public static class SimpleCheckpointVertexMasterCompute extends
-      DefaultMasterCompute {
-    @Override
-    public void initialize() throws InstantiationException,
-        IllegalAccessException {
-      registerAggregator(LongSumAggregator.class.getName(),
-          LongSumAggregator.class);
-    }
-  }
-
-  /**
-   * Executable from the command line.
-   *
-   * @param args Command line args.
-   * @throws Exception
-   */
-  public static void main(String[] args) throws Exception {
-    System.exit(ToolRunner.run(new SimpleCheckpointVertex(), args));
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerComputation.java
new file mode 100644
index 0000000..551a2a2
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerComputation.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Test whether messages can go through a combiner.
+ */
+public class SimpleCombinerComputation extends
+    BasicComputation<LongWritable, IntWritable, FloatWritable, IntWritable> {
+  /** Class logger */
+  private static Logger LOG = Logger.getLogger(SimpleCombinerComputation.class);
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, IntWritable, FloatWritable> vertex,
+      Iterable<IntWritable> messages) throws IOException {
+    if (vertex.getId().equals(new LongWritable(2))) {
+      sendMessage(new LongWritable(1), new IntWritable(101));
+      sendMessage(new LongWritable(1), new IntWritable(102));
+      sendMessage(new LongWritable(1), new IntWritable(103));
+    }
+    if (!vertex.getId().equals(new LongWritable(1))) {
+      vertex.voteToHalt();
+    } else {
+      // Check the messages
+      int sum = 0;
+      int num = 0;
+      for (IntWritable message : messages) {
+        sum += message.get();
+        num++;
+      }
+      LOG.info("TestCombinerVertex: Received a sum of " + sum +
+          " (should have 306 with a single message value)");
+
+      if (num == 1 && sum == 306) {
+        vertex.voteToHalt();
+      }
+    }
+    if (getSuperstep() > 3) {
+      throw new IllegalStateException(
+          "TestCombinerVertex: Vertex 1 failed to receive " +
+          "messages in time");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java
deleted file mode 100644
index f4475ae..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCombinerVertex.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.log4j.Logger;
-
-/**
- * Test whether messages can go through a combiner.
- */
-public class SimpleCombinerVertex extends
-    Vertex<LongWritable, IntWritable, FloatWritable, IntWritable> {
-  /** Class logger */
-  private static Logger LOG = Logger.getLogger(SimpleCombinerVertex.class);
-
-  @Override
-  public void compute(Iterable<IntWritable> messages) {
-    if (getId().equals(new LongWritable(2))) {
-      sendMessage(new LongWritable(1), new IntWritable(101));
-      sendMessage(new LongWritable(1), new IntWritable(102));
-      sendMessage(new LongWritable(1), new IntWritable(103));
-    }
-    if (!getId().equals(new LongWritable(1))) {
-      voteToHalt();
-    } else {
-      // Check the messages
-      int sum = 0;
-      int num = 0;
-      for (IntWritable message : messages) {
-        sum += message.get();
-        num++;
-      }
-      LOG.info("TestCombinerVertex: Received a sum of " + sum +
-          " (should have 306 with a single message value)");
-
-      if (num == 1 && sum == 306) {
-        voteToHalt();
-      }
-    }
-    if (getSuperstep() > 3) {
-      throw new IllegalStateException(
-          "TestCombinerVertex: Vertex 1 failed to receive " +
-          "messages in time");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailComputation.java
new file mode 100644
index 0000000..1582902
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailComputation.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Vertex to allow unit testing of failure detection
+ */
+public class SimpleFailComputation extends BasicComputation<
+    LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+  /** Class logger */
+  private static Logger LOG = Logger.getLogger(SimpleFailComputation.class);
+  /** TODO: Change this behavior to WorkerContext */
+  private static long SUPERSTEP = 0;
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    if (getSuperstep() >= 1) {
+      double sum = 0;
+      for (DoubleWritable message : messages) {
+        sum += message.get();
+      }
+      DoubleWritable vertexValue =
+          new DoubleWritable((0.15f / getTotalNumVertices()) + 0.85f * sum);
+      vertex.setValue(vertexValue);
+      if (getSuperstep() < 30) {
+        if (getSuperstep() == 20) {
+          if (vertex.getId().get() == 10L) {
+            try {
+              Thread.sleep(2000);
+            } catch (InterruptedException e) {
+              LOG.info("Sleep interrupted ", e);
+            }
+            System.exit(1);
+          } else if (getSuperstep() - SUPERSTEP > 10) {
+            return;
+          }
+        }
+        long edges = vertex.getNumEdges();
+        sendMessageToAllEdges(vertex,
+            new DoubleWritable(vertex.getValue().get() / edges));
+      } else {
+        vertex.voteToHalt();
+      }
+      SUPERSTEP = getSuperstep();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
deleted file mode 100644
index c0e206c..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.log4j.Logger;
-
-/**
- * Vertex to allow unit testing of failure detection
- */
-public class SimpleFailVertex extends Vertex<
-    LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
-  /** Class logger */
-  private static Logger LOG = Logger.getLogger(SimpleFailVertex.class);
-  /** TODO: Change this behavior to WorkerContext */
-  private static long SUPERSTEP = 0;
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) {
-    if (getSuperstep() >= 1) {
-      double sum = 0;
-      for (DoubleWritable message : messages) {
-        sum += message.get();
-      }
-      DoubleWritable vertexValue =
-          new DoubleWritable((0.15f / getTotalNumVertices()) + 0.85f * sum);
-      setValue(vertexValue);
-      if (getSuperstep() < 30) {
-        if (getSuperstep() == 20) {
-          if (getId().get() == 10L) {
-            try {
-              Thread.sleep(2000);
-            } catch (InterruptedException e) {
-              LOG.info("Sleep interrupted ", e);
-            }
-            System.exit(1);
-          } else if (getSuperstep() - SUPERSTEP > 10) {
-            return;
-          }
-        }
-        long edges = getNumEdges();
-        sendMessageToAllEdges(
-            new DoubleWritable(getValue().get() / edges));
-      } else {
-        voteToHalt();
-      }
-      SUPERSTEP = getSuperstep();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountComputation.java
new file mode 100644
index 0000000..1513401
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountComputation.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+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 java.io.IOException;
+
+/**
+ * Simple function to return the out degree for each vertex.
+ */
+@Algorithm(
+    name = "Indegree Count"
+)
+public class SimpleInDegreeCountComputation extends BasicComputation<
+  LongWritable, LongWritable, DoubleWritable, DoubleWritable> {
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, LongWritable, DoubleWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    if (getSuperstep() == 0) {
+      Iterable<Edge<LongWritable, DoubleWritable>> edges = vertex.getEdges();
+      for (Edge<LongWritable, DoubleWritable> edge : edges) {
+        sendMessage(edge.getTargetVertexId(), new DoubleWritable(1.0));
+      }
+    } else {
+      long sum = 0;
+      for (DoubleWritable message : messages) {
+        sum++;
+      }
+      LongWritable vertexValue = vertex.getValue();
+      vertexValue.set(sum);
+      vertex.setValue(vertexValue);
+      vertex.voteToHalt();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java
deleted file mode 100644
index 3bcf3f5..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleInDegreeCountVertex.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-
-/**
- * Simple function to return the out degree for each vertex.
- */
-@Algorithm(
-    name = "Indegree Count"
-)
-public class SimpleInDegreeCountVertex extends Vertex<
-  LongWritable, LongWritable, DoubleWritable, DoubleWritable> {
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) {
-    if (getSuperstep() == 0) {
-      Iterable<Edge<LongWritable, DoubleWritable>> edges = getEdges();
-      for (Edge<LongWritable, DoubleWritable> edge : edges) {
-        sendMessage(edge.getTargetVertexId(), new DoubleWritable(1.0));
-      }
-    } else {
-      long sum = 0;
-      for (DoubleWritable message : messages) {
-        sum++;
-      }
-      LongWritable vertexValue = getValue();
-      vertexValue.set(sum);
-      setValue(vertexValue);
-      voteToHalt();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityComputation.java
new file mode 100644
index 0000000..96e0403
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityComputation.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * A simple use of the Identity Vertex for taking care of Long, Double,
+ * Double, Double type Inputformat Good for use with
+ * io.LongDoubleDoubleAdjacencyListVertexInputFormat
+ */
+
+public abstract class SimpleLongDoubleDoubleDoubleIdentityComputation extends
+  IdentityComputation<LongWritable, DoubleWritable,
+  DoubleWritable, DoubleWritable> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityVertex.java
deleted file mode 100644
index c7349d1..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleLongDoubleDoubleDoubleIdentityVertex.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-
-/**
- * A simple use of the Identity Vertex for taking care of Long, Double,
- * Double, Double type Inputformat Good for use with
- * io.LongDoubleDoubleAdjacencyListVertexInputFormat
- */
-
-public abstract class SimpleLongDoubleDoubleDoubleIdentityVertex extends
-  IdentityVertex<LongWritable, DoubleWritable,
-  DoubleWritable, DoubleWritable> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeComputation.java
new file mode 100644
index 0000000..b9b2373
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeComputation.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.aggregators.DoubleOverwriteAggregator;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Demonstrates a computation with a centralized part implemented via a
+ * MasterCompute.
+ */
+public class SimpleMasterComputeComputation extends BasicComputation<
+    LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+  /** Aggregator to get values from the master to the workers */
+  public static final String SMC_AGG = "simplemastercompute.aggregator";
+  /** Logger */
+  private static final Logger LOG =
+      Logger.getLogger(SimpleMasterComputeComputation.class);
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    double oldSum = getSuperstep() == 0 ? 0 : vertex.getValue().get();
+    double newValue = this.<DoubleWritable>getAggregatedValue(SMC_AGG).get();
+    double newSum = oldSum + newValue;
+    vertex.setValue(new DoubleWritable(newSum));
+    SimpleMasterComputeWorkerContext workerContext = getWorkerContext();
+    workerContext.setFinalSum(newSum);
+    LOG.info("Current sum: " + newSum);
+  }
+
+  /**
+   * Worker context used with {@link SimpleMasterComputeComputation}.
+   */
+  public static class SimpleMasterComputeWorkerContext
+      extends WorkerContext {
+    /** Final sum value for verification for local jobs */
+    private static double FINAL_SUM;
+
+    @Override
+    public void preApplication()
+      throws InstantiationException, IllegalAccessException {
+    }
+
+    @Override
+    public void preSuperstep() {
+    }
+
+    @Override
+    public void postSuperstep() {
+    }
+
+    @Override
+    public void postApplication() {
+    }
+
+    public void setFinalSum(double sum) {
+      FINAL_SUM = sum;
+    }
+
+    public static double getFinalSum() {
+      return FINAL_SUM;
+    }
+  }
+
+  /**
+   * MasterCompute used with {@link SimpleMasterComputeComputation}.
+   */
+  public static class SimpleMasterCompute
+      extends DefaultMasterCompute {
+    @Override
+    public void compute() {
+      setAggregatedValue(SMC_AGG,
+          new DoubleWritable(((double) getSuperstep()) / 2 + 1));
+      if (getSuperstep() == 10) {
+        haltComputation();
+      }
+    }
+
+    @Override
+    public void initialize() throws InstantiationException,
+        IllegalAccessException {
+      registerAggregator(SMC_AGG, DoubleOverwriteAggregator.class);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java
deleted file mode 100644
index 8a21ad7..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMasterComputeVertex.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.aggregators.DoubleOverwriteAggregator;
-import org.apache.giraph.master.DefaultMasterCompute;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.worker.WorkerContext;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.log4j.Logger;
-
-/**
- * Demonstrates a computation with a centralized part implemented via a
- * MasterCompute.
- */
-public class SimpleMasterComputeVertex extends Vertex<LongWritable,
-    DoubleWritable, FloatWritable, DoubleWritable> {
-  /** Aggregator to get values from the master to the workers */
-  public static final String SMC_AGG = "simplemastercompute.aggregator";
-  /** Logger */
-  private static final Logger LOG =
-      Logger.getLogger(SimpleMasterComputeVertex.class);
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) {
-    double oldSum = getSuperstep() == 0 ? 0 : getValue().get();
-    double newValue = this.<DoubleWritable>getAggregatedValue(SMC_AGG).get();
-    double newSum = oldSum + newValue;
-    setValue(new DoubleWritable(newSum));
-    SimpleMasterComputeWorkerContext workerContext =
-        (SimpleMasterComputeWorkerContext) getWorkerContext();
-    workerContext.setFinalSum(newSum);
-    LOG.info("Current sum: " + newSum);
-  }
-
-  /**
-   * Worker context used with {@link SimpleMasterComputeVertex}.
-   */
-  public static class SimpleMasterComputeWorkerContext
-      extends WorkerContext {
-    /** Final sum value for verification for local jobs */
-    private static double FINAL_SUM;
-
-    @Override
-    public void preApplication()
-      throws InstantiationException, IllegalAccessException {
-    }
-
-    @Override
-    public void preSuperstep() {
-    }
-
-    @Override
-    public void postSuperstep() {
-    }
-
-    @Override
-    public void postApplication() {
-    }
-
-    public void setFinalSum(double sum) {
-      FINAL_SUM = sum;
-    }
-
-    public static double getFinalSum() {
-      return FINAL_SUM;
-    }
-  }
-
-  /**
-   * MasterCompute used with {@link SimpleMasterComputeVertex}.
-   */
-  public static class SimpleMasterCompute
-      extends DefaultMasterCompute {
-    @Override
-    public void compute() {
-      setAggregatedValue(SMC_AGG,
-          new DoubleWritable(((double) getSuperstep()) / 2 + 1));
-      if (getSuperstep() == 10) {
-        haltComputation();
-      }
-    }
-
-    @Override
-    public void initialize() throws InstantiationException,
-        IllegalAccessException {
-      registerAggregator(SMC_AGG, DoubleOverwriteAggregator.class);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgComputation.java
new file mode 100644
index 0000000..75e69ab
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgComputation.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Test whether messages can be sent and received by vertices.
+ */
+public class SimpleMsgComputation extends
+    BasicComputation<LongWritable, IntWritable, FloatWritable, IntWritable> {
+  /** Class logger */
+  private static Logger LOG = Logger.getLogger(SimpleMsgComputation.class);
+  @Override
+  public void compute(
+      Vertex<LongWritable, IntWritable, FloatWritable> vertex,
+      Iterable<IntWritable> messages) throws IOException {
+    if (vertex.getId().equals(new LongWritable(2))) {
+      sendMessage(new LongWritable(1), new IntWritable(101));
+      sendMessage(new LongWritable(1), new IntWritable(102));
+      sendMessage(new LongWritable(1), new IntWritable(103));
+    }
+    if (!vertex.getId().equals(new LongWritable(1))) {
+      vertex.voteToHalt();
+    } else {
+      /* Check the messages */
+      int sum = 0;
+      for (IntWritable message : messages) {
+        sum += message.get();
+      }
+      LOG.info("compute: Received a sum of " + sum +
+          " (will stop on 306)");
+
+      if (sum == 306) {
+        vertex.voteToHalt();
+      }
+    }
+    if (getSuperstep() > 3) {
+      System.err.println("compute: Vertex 1 failed to receive " +
+          "messages in time");
+      vertex.voteToHalt();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java
deleted file mode 100644
index 024fe9d..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMsgVertex.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.log4j.Logger;
-
-/**
- * Test whether messages can be sent and received by vertices.
- */
-public class SimpleMsgVertex extends
-    Vertex<LongWritable, IntWritable, FloatWritable, IntWritable> {
-  /** Class logger */
-  private static Logger LOG = Logger.getLogger(SimpleMsgVertex.class);
-  @Override
-  public void compute(Iterable<IntWritable> messages) {
-    if (getId().equals(new LongWritable(2))) {
-      sendMessage(new LongWritable(1), new IntWritable(101));
-      sendMessage(new LongWritable(1), new IntWritable(102));
-      sendMessage(new LongWritable(1), new IntWritable(103));
-    }
-    if (!getId().equals(new LongWritable(1))) {
-      voteToHalt();
-    } else {
-      /* Check the messages */
-      int sum = 0;
-      for (IntWritable message : messages) {
-        sum += message.get();
-      }
-      LOG.info("compute: Received a sum of " + sum +
-          " (will stop on 306)");
-
-      if (sum == 306) {
-        voteToHalt();
-      }
-    }
-    if (getSuperstep() > 3) {
-      System.err.println("compute: Vertex 1 failed to receive " +
-          "messages in time");
-      voteToHalt();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphComputation.java
new file mode 100644
index 0000000..fc3b6ad
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphComputation.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+/**
+ * Vertex to allow unit testing of graph mutations.
+ */
+public class SimpleMutateGraphComputation extends BasicComputation<
+    LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+  /** Class logger */
+  private static Logger LOG =
+      Logger.getLogger(SimpleMutateGraphComputation.class);
+  /** Maximum number of ranges for vertex ids */
+  private long maxRanges = 100;
+
+
+  /**
+   * Unless we create a ridiculous number of vertices , we should not
+   * collide within a vertex range defined by this method.
+   *
+   * @param range Range index
+   * @return Starting vertex id of the range
+   */
+  private long rangeVertexIdStart(int range) {
+    return (Long.MAX_VALUE / maxRanges) * range;
+  }
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    SimpleMutateGraphVertexWorkerContext workerContext = getWorkerContext();
+    if (getSuperstep() == 0) {
+      LOG.debug("Reached superstep " + getSuperstep());
+    } else if (getSuperstep() == 1) {
+      // Send messages to vertices that are sure not to exist
+      // (creating them)
+      LongWritable destVertexId =
+          new LongWritable(rangeVertexIdStart(1) + vertex.getId().get());
+      sendMessage(destVertexId, new DoubleWritable(0.0));
+    } else if (getSuperstep() == 2) {
+      LOG.debug("Reached superstep " + getSuperstep());
+    } else if (getSuperstep() == 3) {
+      long vertexCount = workerContext.getVertexCount();
+      if (vertexCount * 2 != getTotalNumVertices()) {
+        throw new IllegalStateException(
+            "Impossible to have " + getTotalNumVertices() +
+            " vertices when should have " + vertexCount * 2 +
+            " on superstep " + getSuperstep());
+      }
+      long edgeCount = workerContext.getEdgeCount();
+      if (edgeCount != getTotalNumEdges()) {
+        throw new IllegalStateException(
+            "Impossible to have " + getTotalNumEdges() +
+            " edges when should have " + edgeCount +
+            " on superstep " + getSuperstep());
+      }
+      // Create vertices that are sure not to exist (doubling vertices)
+      LongWritable vertexIndex =
+          new LongWritable(rangeVertexIdStart(3) + vertex.getId().get());
+      addVertexRequest(vertexIndex, new DoubleWritable(0.0));
+      // Add edges to those remote vertices as well
+      addEdgeRequest(vertexIndex,
+          EdgeFactory.create(vertex.getId(), new FloatWritable(0.0f)));
+    } else if (getSuperstep() == 4) {
+      LOG.debug("Reached superstep " + getSuperstep());
+    } else if (getSuperstep() == 5) {
+      long vertexCount = workerContext.getVertexCount();
+      if (vertexCount * 2 != getTotalNumVertices()) {
+        throw new IllegalStateException(
+            "Impossible to have " + getTotalNumVertices() +
+            " when should have " + vertexCount * 2 +
+            " on superstep " + getSuperstep());
+      }
+      long edgeCount = workerContext.getEdgeCount();
+      if (edgeCount + vertexCount != getTotalNumEdges()) {
+        throw new IllegalStateException(
+            "Impossible to have " + getTotalNumEdges() +
+            " edges when should have " + edgeCount + vertexCount +
+            " on superstep " + getSuperstep());
+      }
+      // Remove the edges created in superstep 3
+      LongWritable vertexIndex =
+          new LongWritable(rangeVertexIdStart(3) + vertex.getId().get());
+      workerContext.increaseEdgesRemoved();
+      removeEdgesRequest(vertexIndex, vertex.getId());
+    } else if (getSuperstep() == 6) {
+      // Remove all the vertices created in superstep 3
+      if (vertex.getId().compareTo(
+          new LongWritable(rangeVertexIdStart(3))) >= 0) {
+        removeVertexRequest(vertex.getId());
+      }
+    } else if (getSuperstep() == 7) {
+      long origEdgeCount = workerContext.getOrigEdgeCount();
+      if (origEdgeCount != getTotalNumEdges()) {
+        throw new IllegalStateException(
+            "Impossible to have " + getTotalNumEdges() +
+            " edges when should have " + origEdgeCount +
+            " on superstep " + getSuperstep());
+      }
+    } else if (getSuperstep() == 8) {
+      long vertexCount = workerContext.getVertexCount();
+      if (vertexCount / 2 != getTotalNumVertices()) {
+        throw new IllegalStateException(
+            "Impossible to have " + getTotalNumVertices() +
+            " vertices when should have " + vertexCount / 2 +
+            " on superstep " + getSuperstep());
+      }
+    } else {
+      vertex.voteToHalt();
+    }
+  }
+
+  /**
+   * Worker context used with {@link SimpleMutateGraphComputation}.
+   */
+  public static class SimpleMutateGraphVertexWorkerContext
+      extends WorkerContext {
+    /** Cached vertex count */
+    private long vertexCount;
+    /** Cached edge count */
+    private long edgeCount;
+    /** Original number of edges */
+    private long origEdgeCount;
+    /** Number of edges removed during superstep */
+    private int edgesRemoved = 0;
+
+    @Override
+    public void preApplication()
+      throws InstantiationException, IllegalAccessException { }
+
+    @Override
+    public void postApplication() { }
+
+    @Override
+    public void preSuperstep() { }
+
+    @Override
+    public void postSuperstep() {
+      vertexCount = getTotalNumVertices();
+      edgeCount = getTotalNumEdges();
+      if (getSuperstep() == 1) {
+        origEdgeCount = edgeCount;
+      }
+      LOG.info("Got " + vertexCount + " vertices, " +
+          edgeCount + " edges on superstep " +
+          getSuperstep());
+      LOG.info("Removed " + edgesRemoved);
+      edgesRemoved = 0;
+    }
+
+    public long getVertexCount() {
+      return vertexCount;
+    }
+
+    public long getEdgeCount() {
+      return edgeCount;
+    }
+
+    public long getOrigEdgeCount() {
+      return origEdgeCount;
+    }
+
+    /**
+     * Increase the number of edges removed by one.
+     */
+    public void increaseEdgesRemoved() {
+      this.edgesRemoved++;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
deleted file mode 100644
index a468491..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.worker.WorkerContext;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.log4j.Logger;
-
-import java.io.IOException;
-
-/**
- * Vertex to allow unit testing of graph mutations.
- */
-public class SimpleMutateGraphVertex extends Vertex<
-    LongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
-  /** Class logger */
-  private static Logger LOG =
-      Logger.getLogger(SimpleMutateGraphVertex.class);
-  /** Maximum number of ranges for vertex ids */
-  private long maxRanges = 100;
-
-
-  /**
-   * Unless we create a ridiculous number of vertices , we should not
-   * collide within a vertex range defined by this method.
-   *
-   * @param range Range index
-   * @return Starting vertex id of the range
-   */
-  private long rangeVertexIdStart(int range) {
-    return (Long.MAX_VALUE / maxRanges) * range;
-  }
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages)
-    throws IOException {
-    SimpleMutateGraphVertexWorkerContext workerContext =
-        (SimpleMutateGraphVertexWorkerContext) getWorkerContext();
-    if (getSuperstep() == 0) {
-      LOG.debug("Reached superstep " + getSuperstep());
-    } else if (getSuperstep() == 1) {
-      // Send messages to vertices that are sure not to exist
-      // (creating them)
-      LongWritable destVertexId =
-          new LongWritable(rangeVertexIdStart(1) + getId().get());
-      sendMessage(destVertexId, new DoubleWritable(0.0));
-    } else if (getSuperstep() == 2) {
-      LOG.debug("Reached superstep " + getSuperstep());
-    } else if (getSuperstep() == 3) {
-      long vertexCount = workerContext.getVertexCount();
-      if (vertexCount * 2 != getTotalNumVertices()) {
-        throw new IllegalStateException(
-            "Impossible to have " + getTotalNumVertices() +
-            " vertices when should have " + vertexCount * 2 +
-            " on superstep " + getSuperstep());
-      }
-      long edgeCount = workerContext.getEdgeCount();
-      if (edgeCount != getTotalNumEdges()) {
-        throw new IllegalStateException(
-            "Impossible to have " + getTotalNumEdges() +
-            " edges when should have " + edgeCount +
-            " on superstep " + getSuperstep());
-      }
-      // Create vertices that are sure not to exist (doubling vertices)
-      LongWritable vertexIndex =
-          new LongWritable(rangeVertexIdStart(3) + getId().get());
-      addVertexRequest(vertexIndex, new DoubleWritable(0.0));
-      // Add edges to those remote vertices as well
-      addEdgeRequest(vertexIndex,
-          EdgeFactory.create(getId(), new FloatWritable(0.0f)));
-    } else if (getSuperstep() == 4) {
-      LOG.debug("Reached superstep " + getSuperstep());
-    } else if (getSuperstep() == 5) {
-      long vertexCount = workerContext.getVertexCount();
-      if (vertexCount * 2 != getTotalNumVertices()) {
-        throw new IllegalStateException(
-            "Impossible to have " + getTotalNumVertices() +
-            " when should have " + vertexCount * 2 +
-            " on superstep " + getSuperstep());
-      }
-      long edgeCount = workerContext.getEdgeCount();
-      if (edgeCount + vertexCount != getTotalNumEdges()) {
-        throw new IllegalStateException(
-            "Impossible to have " + getTotalNumEdges() +
-            " edges when should have " + edgeCount + vertexCount +
-            " on superstep " + getSuperstep());
-      }
-      // Remove the edges created in superstep 3
-      LongWritable vertexIndex =
-          new LongWritable(rangeVertexIdStart(3) + getId().get());
-      workerContext.increaseEdgesRemoved();
-      removeEdgesRequest(vertexIndex, getId());
-    } else if (getSuperstep() == 6) {
-      // Remove all the vertices created in superstep 3
-      if (getId().compareTo(
-          new LongWritable(rangeVertexIdStart(3))) >= 0) {
-        removeVertexRequest(getId());
-      }
-    } else if (getSuperstep() == 7) {
-      long origEdgeCount = workerContext.getOrigEdgeCount();
-      if (origEdgeCount != getTotalNumEdges()) {
-        throw new IllegalStateException(
-            "Impossible to have " + getTotalNumEdges() +
-            " edges when should have " + origEdgeCount +
-            " on superstep " + getSuperstep());
-      }
-    } else if (getSuperstep() == 8) {
-      long vertexCount = workerContext.getVertexCount();
-      if (vertexCount / 2 != getTotalNumVertices()) {
-        throw new IllegalStateException(
-            "Impossible to have " + getTotalNumVertices() +
-            " vertices when should have " + vertexCount / 2 +
-            " on superstep " + getSuperstep());
-      }
-    } else {
-      voteToHalt();
-    }
-  }
-
-  /**
-   * Worker context used with {@link SimpleMutateGraphVertex}.
-   */
-  public static class SimpleMutateGraphVertexWorkerContext
-      extends WorkerContext {
-    /** Cached vertex count */
-    private long vertexCount;
-    /** Cached edge count */
-    private long edgeCount;
-    /** Original number of edges */
-    private long origEdgeCount;
-    /** Number of edges removed during superstep */
-    private int edgesRemoved = 0;
-
-    @Override
-    public void preApplication()
-      throws InstantiationException, IllegalAccessException { }
-
-    @Override
-    public void postApplication() { }
-
-    @Override
-    public void preSuperstep() { }
-
-    @Override
-    public void postSuperstep() {
-      vertexCount = getTotalNumVertices();
-      edgeCount = getTotalNumEdges();
-      if (getSuperstep() == 1) {
-        origEdgeCount = edgeCount;
-      }
-      LOG.info("Got " + vertexCount + " vertices, " +
-          edgeCount + " edges on superstep " +
-          getSuperstep());
-      LOG.info("Removed " + edgesRemoved);
-      edgesRemoved = 0;
-    }
-
-    public long getVertexCount() {
-      return vertexCount;
-    }
-
-    public long getEdgeCount() {
-      return edgeCount;
-    }
-
-    public long getOrigEdgeCount() {
-      return origEdgeCount;
-    }
-
-    /**
-     * Increase the number of edges removed by one.
-     */
-    public void increaseEdgesRemoved() {
-      this.edgesRemoved++;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountComputation.java
new file mode 100644
index 0000000..36dc727
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountComputation.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.giraph.graph.Vertex;
+
+import java.io.IOException;
+
+/**
+ * Simple function to return the out degree for each vertex.
+ */
+@Algorithm(
+    name = "Outdegree Count"
+)
+public class SimpleOutDegreeCountComputation extends BasicComputation<
+  LongWritable, LongWritable, DoubleWritable, DoubleWritable> {
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, LongWritable, DoubleWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    LongWritable vertexValue = vertex.getValue();
+    vertexValue.set(vertex.getNumEdges());
+    vertex.setValue(vertexValue);
+    vertex.voteToHalt();
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java
deleted file mode 100644
index c830fa2..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleOutDegreeCountVertex.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.examples;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.giraph.graph.Vertex;
-
-
-/**
- * Simple function to return the out degree for each vertex.
- */
-@Algorithm(
-    name = "Outdegree Count"
-)
-public class SimpleOutDegreeCountVertex extends Vertex<
-  LongWritable, LongWritable,
-  DoubleWritable, DoubleWritable> {
-
-  @Override
-  public void compute(Iterable<DoubleWritable> messages) {
-    LongWritable vertexValue = getValue();
-    vertexValue.set(getNumEdges());
-    setValue(vertexValue);
-    voteToHalt();
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java
new file mode 100644
index 0000000..d053bb3
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.examples;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.List;
+import org.apache.giraph.aggregators.DoubleMaxAggregator;
+import org.apache.giraph.aggregators.DoubleMinAggregator;
+import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.graph.BasicComputation;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.VertexReader;
+import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.giraph.master.DefaultMasterCompute;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.log4j.Logger;
+
+/**
+ * Demonstrates the basic Pregel PageRank implementation.
+ */
+@Algorithm(
+    name = "Page rank"
+)
+public class SimplePageRankComputation extends BasicComputation<LongWritable,
+    DoubleWritable, FloatWritable, DoubleWritable> {
+  /** Number of supersteps for this test */
+  public static final int MAX_SUPERSTEPS = 30;
+  /** Logger */
+  private static final Logger LOG =
+      Logger.getLogger(SimplePageRankComputation.class);
+  /** Sum aggregator name */
+  private static String SUM_AGG = "sum";
+  /** Min aggregator name */
+  private static String MIN_AGG = "min";
+  /** Max aggregator name */
+  private static String MAX_AGG = "max";
+
+  @Override
+  public void compute(
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex,
+      Iterable<DoubleWritable> messages) throws IOException {
+    if (getSuperstep() >= 1) {
+      double sum = 0;
+      for (DoubleWritable message : messages) {
+        sum += message.get();
+      }
+      DoubleWritable vertexValue =
+          new DoubleWritable((0.15f / getTotalNumVertices()) + 0.85f * sum);
+      vertex.setValue(vertexValue);
+      aggregate(MAX_AGG, vertexValue);
+      aggregate(MIN_AGG, vertexValue);
+      aggregate(SUM_AGG, new LongWritable(1));
+      LOG.info(vertex.getId() + ": PageRank=" + vertexValue +
+          " max=" + getAggregatedValue(MAX_AGG) +
+          " min=" + getAggregatedValue(MIN_AGG));
+    }
+
+    if (getSuperstep() < MAX_SUPERSTEPS) {
+      long edges = vertex.getNumEdges();
+      sendMessageToAllEdges(vertex,
+          new DoubleWritable(vertex.getValue().get() / edges));
+    } else {
+      vertex.voteToHalt();
+    }
+  }
+
+  /**
+   * Worker context used with {@link SimplePageRankComputation}.
+   */
+  public static class SimplePageRankWorkerContext extends
+      WorkerContext {
+    /** Final max value for verification for local jobs */
+    private static double FINAL_MAX;
+    /** Final min value for verification for local jobs */
+    private static double FINAL_MIN;
+    /** Final sum value for verification for local jobs */
+    private static long FINAL_SUM;
+
+    public static double getFinalMax() {
+      return FINAL_MAX;
+    }
+
+    public static double getFinalMin() {
+      return FINAL_MIN;
+    }
+
+    public static long getFinalSum() {
+      return FINAL_SUM;
+    }
+
+    @Override
+    public void preApplication()
+      throws InstantiationException, IllegalAccessException {
+    }
+
+    @Override
+    public void postApplication() {
+      FINAL_SUM = this.<LongWritable>getAggregatedValue(SUM_AGG).get();
+      FINAL_MAX = this.<DoubleWritable>getAggregatedValue(MAX_AGG).get();
+      FINAL_MIN = this.<DoubleWritable>getAggregatedValue(MIN_AGG).get();
+
+      LOG.info("aggregatedNumVertices=" + FINAL_SUM);
+      LOG.info("aggregatedMaxPageRank=" + FINAL_MAX);
+      LOG.info("aggregatedMinPageRank=" + FINAL_MIN);
+    }
+
+    @Override
+    public void preSuperstep() {
+      if (getSuperstep() >= 3) {
+        LOG.info("aggregatedNumVertices=" +
+            getAggregatedValue(SUM_AGG) +
+            " NumVertices=" + getTotalNumVertices());
+        if (this.<LongWritable>getAggregatedValue(SUM_AGG).get() !=
+            getTotalNumVertices()) {
+          throw new RuntimeException("wrong value of SumAggreg: " +
+              getAggregatedValue(SUM_AGG) + ", should be: " +
+              getTotalNumVertices());
+        }
+        DoubleWritable maxPagerank = getAggregatedValue(MAX_AGG);
+        LOG.info("aggregatedMaxPageRank=" + maxPagerank.get());
+        DoubleWritable minPagerank = getAggregatedValue(MIN_AGG);
+        LOG.info("aggregatedMinPageRank=" + minPagerank.get());
+      }
+    }
+
+    @Override
+    public void postSuperstep() { }
+  }
+
+  /**
+   * Master compute associated with {@link SimplePageRankComputation}.
+   * It registers required aggregators.
+   */
+  public static class SimplePageRankMasterCompute extends
+      DefaultMasterCompute {
+    @Override
+    public void initialize() throws InstantiationException,
+        IllegalAccessException {
+      registerAggregator(SUM_AGG, LongSumAggregator.class);
+      registerPersistentAggregator(MIN_AGG, DoubleMinAggregator.class);
+      registerPersistentAggregator(MAX_AGG, DoubleMaxAggregator.class);
+    }
+  }
+
+  /**
+   * Simple VertexReader that supports {@link SimplePageRankComputation}
+   */
+  public static class SimplePageRankVertexReader extends
+      GeneratedVertexReader<LongWritable, DoubleWritable, FloatWritable> {
+    /** Class logger */
+    private static final Logger LOG =
+        Logger.getLogger(SimplePageRankVertexReader.class);
+
+    @Override
+    public boolean nextVertex() {
+      return totalRecords > recordsRead;
+    }
+
+    @Override
+    public Vertex<LongWritable, DoubleWritable,
+        FloatWritable> getCurrentVertex() throws IOException {
+      Vertex<LongWritable, DoubleWritable, FloatWritable> vertex =
+          getConf().createVertex();
+      LongWritable vertexId = new LongWritable(
+          (inputSplit.getSplitIndex() * totalRecords) + recordsRead);
+      DoubleWritable vertexValue = new DoubleWritable(vertexId.get() * 10d);
+      long targetVertexId =
+          (vertexId.get() + 1) %
+          (inputSplit.getNumSplits() * totalRecords);
+      float edgeValue = vertexId.get() * 100f;
+      List<Edge<LongWritable, FloatWritable>> edges = Lists.newLinkedList();
+      edges.add(EdgeFactory.create(new LongWritable(targetVertexId),
+          new FloatWritable(edgeValue)));
+      vertex.initialize(vertexId, vertexValue, edges);
+      ++recordsRead;
+      if (LOG.isInfoEnabled()) {
+        LOG.info("next: Return vertexId=" + vertex.getId().get() +
+            ", vertexValue=" + vertex.getValue() +
+            ", targetVertexId=" + targetVertexId + ", edgeValue=" + edgeValue);
+      }
+      return vertex;
+    }
+  }
+
+  /**
+   * Simple VertexInputFormat that supports {@link SimplePageRankComputation}
+   */
+  public static class SimplePageRankVertexInputFormat extends
+    GeneratedVertexInputFormat<LongWritable, DoubleWritable, FloatWritable> {
+    @Override
+    public VertexReader<LongWritable, DoubleWritable,
+    FloatWritable> createVertexReader(InputSplit split,
+      TaskAttemptContext context)
+      throws IOException {
+      return new SimplePageRankVertexReader();
+    }
+  }
+
+  /**
+   * Simple VertexOutputFormat that supports {@link SimplePageRankComputation}
+   */
+  public static class SimplePageRankVertexOutputFormat extends
+      TextVertexOutputFormat<LongWritable, DoubleWritable, FloatWritable> {
+    @Override
+    public TextVertexWriter createVertexWriter(TaskAttemptContext context)
+      throws IOException, InterruptedException {
+      return new SimplePageRankVertexWriter();
+    }
+
+    /**
+     * Simple VertexWriter that supports {@link SimplePageRankComputation}
+     */
+    public class SimplePageRankVertexWriter extends TextVertexWriter {
+      @Override
+      public void writeVertex(
+          Vertex<LongWritable, DoubleWritable, FloatWritable> vertex)
+        throws IOException, InterruptedException {
+        getRecordWriter().write(
+            new Text(vertex.getId().toString()),
+            new Text(vertex.getValue().toString()));
+      }
+    }
+  }
+}


[10/12] GIRAPH-667: Decouple Vertex data and Computation, make Computation and Combiner classes switchable (majakabiljo)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
index 10e4975..621bb14 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
@@ -20,11 +20,11 @@ package org.apache.giraph.conf;
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.aggregators.TextAggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
 import org.apache.giraph.graph.DefaultVertexValueFactory;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.filters.DefaultEdgeInputFilter;
@@ -36,11 +36,9 @@ import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.master.MasterCompute;
-import org.apache.giraph.partition.DefaultPartitionContext;
 import org.apache.giraph.partition.GraphPartitionerFactory;
 import org.apache.giraph.partition.HashPartitionerFactory;
 import org.apache.giraph.partition.Partition;
-import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.partition.SimplePartition;
 import org.apache.giraph.utils.ReflectionUtils;
 import org.apache.giraph.worker.DefaultWorkerContext;
@@ -57,22 +55,25 @@ import java.util.List;
  * @param <I> Vertex ID class
  * @param <V> Vertex Value class
  * @param <E> Edge class
- * @param <M> Message class
  */
 @SuppressWarnings("unchecked")
 public class GiraphClasses<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     implements GiraphConstants {
-  /** Vertex class - cached for fast access */
-  protected Class<? extends Vertex<I, V, E, M>> vertexClass;
+  /** Computation class - cached for fast access */
+  protected Class<? extends
+      Computation<I, V, E, ? extends Writable, ? extends Writable>>
+  computationClass;
   /** Vertex id class - cached for fast access */
   protected Class<I> vertexIdClass;
   /** Vertex value class - cached for fast access */
   protected Class<V> vertexValueClass;
   /** Edge value class - cached for fast access */
   protected Class<E> edgeValueClass;
-  /** Message value class - cached for fast access */
-  protected Class<M> messageValueClass;
+  /** Incoming message value class - cached for fast access */
+  protected Class<? extends Writable> incomingMessageValueClass;
+  /** Outgoing message value class - cached for fast access */
+  protected Class<? extends Writable> outgoingMessageValueClass;
   /** Vertex edges class - cached for fast access */
   protected Class<? extends OutEdges<I, E>> outEdgesClass;
   /** Input vertex edges class - cached for fast access */
@@ -82,7 +83,7 @@ public class GiraphClasses<I extends WritableComparable,
   protected Class<? extends VertexValueFactory<V>> vertexValueFactoryClass;
 
   /** Graph partitioner factory class - cached for fast access */
-  protected Class<? extends GraphPartitionerFactory<I, V, E, M>>
+  protected Class<? extends GraphPartitionerFactory<I, V, E>>
   graphPartitionerFactoryClass;
 
   /** Vertex input format class - cached for fast access */
@@ -98,24 +99,22 @@ public class GiraphClasses<I extends WritableComparable,
   /** Aggregator writer class - cached for fast access */
   protected Class<? extends AggregatorWriter> aggregatorWriterClass;
   /** Combiner class - cached for fast access */
-  protected Class<? extends Combiner<I, M>> combinerClass;
+  protected Class<? extends Combiner<I, ? extends Writable>> combinerClass;
 
   /** Vertex resolver class - cached for fast access */
-  protected Class<? extends VertexResolver<I, V, E, M>> vertexResolverClass;
-  /** Partition context class - cached for fast access */
-  protected Class<? extends PartitionContext> partitionContextClass;
+  protected Class<? extends VertexResolver<I, V, E>> vertexResolverClass;
   /** Worker context class - cached for fast access */
   protected Class<? extends WorkerContext> workerContextClass;
   /** Master compute class - cached for fast access */
   protected Class<? extends MasterCompute> masterComputeClass;
 
   /** Partition class - cached for fast accesss */
-  protected Class<? extends Partition<I, V, E, M>> partitionClass;
+  protected Class<? extends Partition<I, V, E>> partitionClass;
 
   /** Edge Input Filter class */
   protected Class<? extends EdgeInputFilter<I, E>> edgeInputFilterClass;
   /** Vertex Input Filter class */
-  protected Class<? extends VertexInputFilter<I, V, E, M>>
+  protected Class<? extends VertexInputFilter<I, V, E>>
   vertexInputFilterClass;
 
   /**
@@ -131,19 +130,18 @@ public class GiraphClasses<I extends WritableComparable,
     vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>) (Object)
         DefaultVertexValueFactory.class;
     graphPartitionerFactoryClass =
-        (Class<? extends GraphPartitionerFactory<I, V, E, M>>) (Object)
+        (Class<? extends GraphPartitionerFactory<I, V, E>>) (Object)
             HashPartitionerFactory.class;
     aggregatorWriterClass = TextAggregatorWriter.class;
-    vertexResolverClass = (Class<? extends VertexResolver<I, V, E, M>>)
+    vertexResolverClass = (Class<? extends VertexResolver<I, V, E>>)
         (Object) DefaultVertexResolver.class;
-    partitionContextClass = DefaultPartitionContext.class;
     workerContextClass = DefaultWorkerContext.class;
     masterComputeClass = DefaultMasterCompute.class;
-    partitionClass = (Class<? extends Partition<I, V, E, M>>) (Object)
+    partitionClass = (Class<? extends Partition<I, V, E>>) (Object)
         SimplePartition.class;
     edgeInputFilterClass = (Class<? extends EdgeInputFilter<I, E>>)
         (Object) DefaultEdgeInputFilter.class;
-    vertexInputFilterClass = (Class<? extends VertexInputFilter<I, V, E, M>>)
+    vertexInputFilterClass = (Class<? extends VertexInputFilter<I, V, E>>)
         (Object) DefaultVertexInputFilter.class;
   }
 
@@ -163,13 +161,17 @@ public class GiraphClasses<I extends WritableComparable,
    */
   private void readFromConf(Configuration conf) {
     // set pre-validated generic parameter types into Configuration
-    vertexClass = (Class<? extends Vertex<I, V, E, M>>) VERTEX_CLASS.get(conf);
-    List<Class<?>> classList = ReflectionUtils.getTypeArguments(Vertex.class,
-        vertexClass);
+    computationClass =
+        (Class<? extends
+            Computation<I, V, E, ? extends Writable, ? extends Writable>>)
+            COMPUTATION_CLASS.get(conf);
+    List<Class<?>> classList =
+        ReflectionUtils.getTypeArguments(Computation.class, computationClass);
     vertexIdClass = (Class<I>) classList.get(0);
     vertexValueClass = (Class<V>) classList.get(1);
     edgeValueClass = (Class<E>) classList.get(2);
-    messageValueClass = (Class<M>) classList.get(3);
+    incomingMessageValueClass = (Class<? extends Writable>) classList.get(3);
+    outgoingMessageValueClass = (Class<? extends Writable>) classList.get(4);
 
     outEdgesClass = (Class<? extends OutEdges<I, E>>)
         VERTEX_EDGES_CLASS.get(conf);
@@ -179,7 +181,7 @@ public class GiraphClasses<I extends WritableComparable,
         VERTEX_VALUE_FACTORY_CLASS.get(conf);
 
     graphPartitionerFactoryClass =
-        (Class<? extends GraphPartitionerFactory<I, V, E, M>>)
+        (Class<? extends GraphPartitionerFactory<I, V, E>>)
             GRAPH_PARTITIONER_FACTORY_CLASS.get(conf);
 
     vertexInputFormatClass = (Class<? extends VertexInputFormat<I, V, E>>)
@@ -190,29 +192,30 @@ public class GiraphClasses<I extends WritableComparable,
         EDGE_INPUT_FORMAT_CLASS.get(conf);
 
     aggregatorWriterClass = AGGREGATOR_WRITER_CLASS.get(conf);
-    combinerClass = (Class<? extends Combiner<I, M>>)
+    combinerClass = (Class<? extends Combiner<I, ? extends Writable>>)
         VERTEX_COMBINER_CLASS.get(conf);
-    vertexResolverClass = (Class<? extends VertexResolver<I, V, E, M>>)
+    vertexResolverClass = (Class<? extends VertexResolver<I, V, E>>)
         VERTEX_RESOLVER_CLASS.get(conf);
-    partitionContextClass = PARTITION_CONTEXT_CLASS.get(conf);
     workerContextClass = WORKER_CONTEXT_CLASS.get(conf);
     masterComputeClass =  MASTER_COMPUTE_CLASS.get(conf);
-    partitionClass = (Class<? extends Partition<I, V, E, M>>)
+    partitionClass = (Class<? extends Partition<I, V, E>>)
         PARTITION_CLASS.get(conf);
 
     edgeInputFilterClass = (Class<? extends EdgeInputFilter<I, E>>)
         EDGE_INPUT_FILTER_CLASS.get(conf);
-    vertexInputFilterClass = (Class<? extends VertexInputFilter<I, V, E, M>>)
+    vertexInputFilterClass = (Class<? extends VertexInputFilter<I, V, E>>)
         VERTEX_INPUT_FILTER_CLASS.get(conf);
   }
 
   /**
-   * Get Vertex class
+   * Get Computation class
    *
-   * @return Vertex class.
+   * @return Computation class.
    */
-  public Class<? extends Vertex<I, V, E, M>> getVertexClass() {
-    return vertexClass;
+  public Class<? extends
+      Computation<I, V, E, ? extends Writable, ? extends Writable>>
+  getComputationClass() {
+    return computationClass;
   }
 
   /**
@@ -243,12 +246,23 @@ public class GiraphClasses<I extends WritableComparable,
   }
 
   /**
-   * Get Message Value class
+   * Get incoming Message Value class - messages which have been sent in the
+   * previous superstep and are processed in the current one
    *
    * @return Message Value class
    */
-  public Class<M> getMessageValueClass() {
-    return messageValueClass;
+  public Class<? extends Writable> getIncomingMessageValueClass() {
+    return incomingMessageValueClass;
+  }
+
+  /**
+   * Get outgoing Message Value class - messages which are going to be sent
+   * during current superstep
+   *
+   * @return Message Value class
+   */
+  public Class<? extends Writable> getOutgoingMessageValueClass() {
+    return outgoingMessageValueClass;
   }
 
   /**
@@ -283,7 +297,7 @@ public class GiraphClasses<I extends WritableComparable,
    *
    * @return GraphPartitionerFactory
    */
-  public Class<? extends GraphPartitionerFactory<I, V, E, M>>
+  public Class<? extends GraphPartitionerFactory<I, V, E>>
   getGraphPartitionerFactoryClass() {
     return graphPartitionerFactoryClass;
   }
@@ -293,7 +307,7 @@ public class GiraphClasses<I extends WritableComparable,
     return edgeInputFilterClass;
   }
 
-  public Class<? extends VertexInputFilter<I, V, E, M>>
+  public Class<? extends VertexInputFilter<I, V, E>>
   getVertexInputFilterClass() {
     return vertexInputFilterClass;
   }
@@ -386,7 +400,7 @@ public class GiraphClasses<I extends WritableComparable,
    *
    * @return Combiner
    */
-  public Class<? extends Combiner<I, M>> getCombinerClass() {
+  public Class<? extends Combiner<I, ? extends Writable>> getCombinerClass() {
     return combinerClass;
   }
 
@@ -404,29 +418,11 @@ public class GiraphClasses<I extends WritableComparable,
    *
    * @return VertexResolver
    */
-  public Class<? extends VertexResolver<I, V, E, M>> getVertexResolverClass() {
+  public Class<? extends VertexResolver<I, V, E>> getVertexResolverClass() {
     return vertexResolverClass;
   }
 
   /**
-   * Check if PartitionContext is set
-   *
-   * @return true if PartitionContext is set
-   */
-  public boolean hasPartitionContextClass() {
-    return partitionContextClass != null;
-  }
-
-  /**
-   * Get PartitionContext used
-   *
-   * @return PartitionContext
-   */
-  public Class<? extends PartitionContext> getPartitionContextClass() {
-    return partitionContextClass;
-  }
-
-  /**
    * Check if WorkerContext is set
    *
    * @return true if WorkerContext is set
@@ -476,19 +472,24 @@ public class GiraphClasses<I extends WritableComparable,
    *
    * @return Partition
    */
-  public Class<? extends Partition<I, V, E, M>> getPartitionClass() {
+  public Class<? extends Partition<I, V, E>> getPartitionClass() {
     return partitionClass;
   }
 
   /**
-   * Set Vertex class held
+   * Set Computation class held, and update message types
    *
-   * @param vertexClass Vertex class to set
+   * @param computationClass Computation class to set
    * @return this
    */
-  public GiraphClasses setVertexClass(
-      Class<? extends Vertex<I, V, E, M>> vertexClass) {
-    this.vertexClass = vertexClass;
+  public GiraphClasses setComputationClass(Class<? extends
+      Computation<I, V, E, ? extends Writable, ? extends Writable>>
+      computationClass) {
+    this.computationClass = computationClass;
+    List<Class<?>> classList =
+        ReflectionUtils.getTypeArguments(Computation.class, computationClass);
+    incomingMessageValueClass = (Class<? extends Writable>) classList.get(3);
+    outgoingMessageValueClass = (Class<? extends Writable>) classList.get(4);
     return this;
   }
 
@@ -526,13 +527,28 @@ public class GiraphClasses<I extends WritableComparable,
   }
 
   /**
-   * Set Message Value class held
+   * Set incoming Message Value class held - messages which have been sent in
+   * the previous superstep and are processed in the current one
+   *
+   * @param incomingMessageValueClass Message Value class to set
+   * @return this
+   */
+  public GiraphClasses setIncomingMessageValueClass(
+      Class<? extends Writable> incomingMessageValueClass) {
+    this.incomingMessageValueClass = incomingMessageValueClass;
+    return this;
+  }
+
+  /**
+   * Set outgoing Message Value class held - messages which are going to be sent
+   * during current superstep
    *
-   * @param messageValueClass Message Value class to set
+   * @param outgoingMessageValueClass Message Value class to set
    * @return this
    */
-  public GiraphClasses setMessageValueClass(Class<M> messageValueClass) {
-    this.messageValueClass = messageValueClass;
+  public GiraphClasses setOutgoingMessageValueClass(
+      Class<? extends Writable> outgoingMessageValueClass) {
+    this.outgoingMessageValueClass = outgoingMessageValueClass;
     return this;
   }
 
@@ -583,7 +599,7 @@ public class GiraphClasses<I extends WritableComparable,
    * @return this
    */
   public GiraphClasses setGraphPartitionerFactoryClass(
-      Class<? extends GraphPartitionerFactory<I, V, E, M>> klass) {
+      Class<? extends GraphPartitionerFactory<I, V, E>> klass) {
     this.graphPartitionerFactoryClass = klass;
     return this;
   }
@@ -643,7 +659,7 @@ public class GiraphClasses<I extends WritableComparable,
    * @return this
    */
   public GiraphClasses setCombinerClass(
-      Class<? extends Combiner<I, M>> combinerClass) {
+      Class<? extends Combiner<I, ? extends Writable>> combinerClass) {
     this.combinerClass = combinerClass;
     return this;
   }
@@ -655,24 +671,12 @@ public class GiraphClasses<I extends WritableComparable,
    * @return this
    */
   public GiraphClasses setVertexResolverClass(
-      Class<? extends VertexResolver<I, V, E, M>> vertexResolverClass) {
+      Class<? extends VertexResolver<I, V, E>> vertexResolverClass) {
     this.vertexResolverClass = vertexResolverClass;
     return this;
   }
 
   /**
-   * Set PartitionContext used
-   *
-   * @param partitionContextClass PartitionContext class to set
-   * @return this
-   */
-  public GiraphClasses setPartitionContextClass(
-      Class<? extends PartitionContext> partitionContextClass) {
-    this.partitionContextClass = partitionContextClass;
-    return this;
-  }
-
-  /**
    * Set WorkerContext used
    *
    * @param workerContextClass WorkerContext class to set
@@ -703,7 +707,7 @@ public class GiraphClasses<I extends WritableComparable,
    * @return this
    */
   public GiraphClasses setPartitionClass(
-      Class<? extends Partition<I, V, E, M>> partitionClass) {
+      Class<? extends Partition<I, V, E>> partitionClass) {
     this.partitionClass = partitionClass;
     return this;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
index 754fad9..58a3f01 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
@@ -20,9 +20,9 @@ package org.apache.giraph.conf;
 
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.edge.ReuseObjectsOutEdges;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
@@ -35,7 +35,6 @@ import org.apache.giraph.master.MasterCompute;
 import org.apache.giraph.master.MasterObserver;
 import org.apache.giraph.partition.GraphPartitionerFactory;
 import org.apache.giraph.partition.Partition;
-import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.partition.ReusesObjectsPartition;
 import org.apache.giraph.worker.WorkerContext;
 import org.apache.giraph.worker.WorkerObserver;
@@ -79,22 +78,22 @@ public class GiraphConfiguration extends Configuration
   }
 
   /**
-   * Get the user's subclassed {@link org.apache.giraph.graph.Vertex}
+   * Get the user's subclassed {@link Computation}
    *
-   * @return User's vertex class
+   * @return User's computation class
    */
-  public Class<? extends Vertex> getVertexClass() {
-    return VERTEX_CLASS.get(this);
+  public Class<? extends Computation> getComputationClass() {
+    return COMPUTATION_CLASS.get(this);
   }
 
   /**
-   * Set the vertex class (required)
+   * Set the computation class (required)
    *
-   * @param vertexClass Runs vertex computation
+   * @param computationClass Runs vertex computation
    */
-  public final void setVertexClass(
-      Class<? extends Vertex> vertexClass) {
-    VERTEX_CLASS.set(this, vertexClass);
+  public void setComputationClass(
+      Class<? extends Computation> computationClass) {
+    COMPUTATION_CLASS.set(this, computationClass);
   }
 
   /**
@@ -407,7 +406,7 @@ public class GiraphConfiguration extends Configuration
    *
    * @param vertexCombinerClass Determines how vertex messages are combined
    */
-  public final void setCombinerClass(
+  public void setCombinerClass(
       Class<? extends Combiner> vertexCombinerClass) {
     VERTEX_COMBINER_CLASS.set(this, vertexCombinerClass);
   }
@@ -452,17 +451,6 @@ public class GiraphConfiguration extends Configuration
   }
 
   /**
-   * Set the partition context class (optional)
-   *
-   * @param partitionContextClass Determines what code is executed for each
-   *        partition before and after each superstep
-   */
-  public final void setPartitionContextClass(
-      Class<? extends PartitionContext> partitionContextClass) {
-    PARTITION_CONTEXT_CLASS.set(this, partitionContextClass);
-  }
-
-  /**
    * Set the worker context class (optional)
    *
    * @param workerContextClass Determines what code is executed on a each

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
index bbf50e5..2d0f59c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
@@ -20,11 +20,11 @@ package org.apache.giraph.conf;
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.aggregators.TextAggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
 import org.apache.giraph.graph.DefaultVertexValueFactory;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
@@ -39,11 +39,9 @@ import org.apache.giraph.job.GiraphJobObserver;
 import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.master.MasterCompute;
 import org.apache.giraph.master.MasterObserver;
-import org.apache.giraph.partition.DefaultPartitionContext;
 import org.apache.giraph.partition.GraphPartitionerFactory;
 import org.apache.giraph.partition.HashPartitionerFactory;
 import org.apache.giraph.partition.Partition;
-import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.partition.SimplePartition;
 import org.apache.giraph.worker.DefaultWorkerContext;
 import org.apache.giraph.worker.WorkerContext;
@@ -62,9 +60,10 @@ public interface GiraphConstants {
   /** 1KB in bytes */
   int ONE_KB = 1024;
 
-  /** Vertex class - required */
-  ClassConfOption<Vertex> VERTEX_CLASS =
-      ClassConfOption.create("giraph.vertexClass", null, Vertex.class);
+  /** Computation class - required */
+  ClassConfOption<Computation> COMPUTATION_CLASS =
+      ClassConfOption.create("giraph.computationClass", null,
+          Computation.class);
   /** Vertex value factory class - optional */
   ClassConfOption<VertexValueFactory> VERTEX_VALUE_FACTORY_CLASS =
       ClassConfOption.create("giraph.vertexValueFactoryClass",
@@ -184,13 +183,14 @@ public interface GiraphConstants {
   /** Edge value class */
   ClassConfOption<Writable> EDGE_VALUE_CLASS =
       ClassConfOption.create("giraph.edgeValueClass", null, Writable.class);
-  /** Message value class */
-  ClassConfOption<Writable> MESSAGE_VALUE_CLASS =
-      ClassConfOption.create("giraph.messageValueClass", null, Writable.class);
-  /** Partition context class */
-  ClassConfOption<PartitionContext> PARTITION_CONTEXT_CLASS =
-      ClassConfOption.create("giraph.partitionContextClass",
-          DefaultPartitionContext.class, PartitionContext.class);
+  /** Incoming message value class */
+  ClassConfOption<Writable> INCOMING_MESSAGE_VALUE_CLASS =
+      ClassConfOption.create("giraph.incomingMessageValueClass", null,
+          Writable.class);
+  /** Outgoing message value class */
+  ClassConfOption<Writable> OUTGOING_MESSAGE_VALUE_CLASS =
+      ClassConfOption.create("giraph.outgoingMessageValueClass", null,
+          Writable.class);
   /** Worker context class */
   ClassConfOption<WorkerContext> WORKER_CONTEXT_CLASS =
       ClassConfOption.create("giraph.workerContextClass",

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
index e4351a2..dbffbc7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfigurable.java
@@ -27,23 +27,20 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public interface ImmutableClassesGiraphConfigurable<
-    I extends WritableComparable, V extends Writable, E extends Writable,
-    M extends Writable> {
+    I extends WritableComparable, V extends Writable, E extends Writable> {
   /**
    * Set the configuration to be used by this object.
    *
    * @param configuration Set configuration
    */
-  void setConf(ImmutableClassesGiraphConfiguration<I, V, E, M>
-                   configuration);
+  void setConf(ImmutableClassesGiraphConfiguration<I, V, E> configuration);
 
   /**
    * Return the configuration used by this object.
    *
    * @return Set configuration
    */
-  ImmutableClassesGiraphConfiguration<I, V, E, M> getConf();
+  ImmutableClassesGiraphConfiguration<I, V, E> getConf();
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
index a9add4f..aa52498 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
@@ -20,11 +20,11 @@ package org.apache.giraph.conf;
 
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.edge.ReusableEdge;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.graph.VertexValueFactory;
@@ -43,9 +43,9 @@ import org.apache.giraph.io.superstep_output.SynchronizedSuperstepOutput;
 import org.apache.giraph.job.GiraphJobObserver;
 import org.apache.giraph.master.MasterCompute;
 import org.apache.giraph.master.MasterObserver;
+import org.apache.giraph.master.SuperstepClasses;
 import org.apache.giraph.partition.GraphPartitionerFactory;
 import org.apache.giraph.partition.Partition;
-import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.utils.ExtendedByteArrayDataInput;
 import org.apache.giraph.utils.ExtendedByteArrayDataOutput;
 import org.apache.giraph.utils.ExtendedDataInput;
@@ -70,11 +70,10 @@ import org.apache.hadoop.util.Progressable;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("unchecked")
 public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     extends GiraphConfiguration {
   /** Holder for all the classes */
   private final GiraphClasses classes;
@@ -95,7 +94,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    */
   public ImmutableClassesGiraphConfiguration(Configuration conf) {
     super(conf);
-    classes = new GiraphClasses<I, V, E, M>(conf);
+    classes = new GiraphClasses<I, V, E>(conf);
     useUnsafeSerialization = USE_UNSAFE_SERIALIZATION.get(this);
     try {
       vertexValueFactory = (VertexValueFactory<V>)
@@ -113,23 +112,6 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   }
 
   /**
-   * Create a new ImmutableClassesGiraphConfiguration. This is a convenience
-   * method to make it easier to deal with generics.
-   *
-   * @param conf Configuration to read
-   * @param <I> Vertex ID
-   * @param <V> Vertex Value
-   * @param <E> Edge Value
-   * @param <M> Message Value
-   * @return new ImmutableClassesGiraphConfiguration
-   */
-  public static <I extends WritableComparable,
-      V extends Writable, E extends Writable, M extends Writable>
-  ImmutableClassesGiraphConfiguration<I, V, E, M> create(Configuration conf) {
-    return new ImmutableClassesGiraphConfiguration<I, V, E, M>(conf);
-  }
-
-  /**
    * Configure an object with this instance if the object is configurable.
    * @param obj Object
    */
@@ -162,7 +144,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return VertexInputFilter class
    */
-  public Class<? extends VertexInputFilter<I, V, E, M>>
+  public Class<? extends VertexInputFilter<I, V, E>>
   getVertexInputFilterClass() {
     return classes.getVertexInputFilterClass();
   }
@@ -181,7 +163,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return User's graph partitioner
    */
-  public Class<? extends GraphPartitionerFactory<I, V, E, M>>
+  public Class<? extends GraphPartitionerFactory<I, V, E>>
   getGraphPartitionerClass() {
     return classes.getGraphPartitionerFactoryClass();
   }
@@ -191,8 +173,8 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return Instantiated user graph partitioner class
    */
-  public GraphPartitionerFactory<I, V, E, M> createGraphPartitioner() {
-    Class<? extends GraphPartitionerFactory<I, V, E, M>> klass =
+  public GraphPartitionerFactory<I, V, E> createGraphPartitioner() {
+    Class<? extends GraphPartitionerFactory<I, V, E>> klass =
         classes.getGraphPartitionerFactoryClass();
     return ReflectionUtils.newInstance(klass, this);
   }
@@ -380,17 +362,18 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return User's combiner class
    */
-  public Class<? extends Combiner<I, M>> getCombinerClass() {
+  public Class<? extends Combiner<I, ? extends Writable>> getCombinerClass() {
     return classes.getCombinerClass();
   }
 
   /**
    * Create a user combiner class
    *
+   * @param <M> Message data
    * @return Instantiated user combiner class
    */
   @SuppressWarnings("rawtypes")
-  public Combiner<I, M> createCombiner() {
+  public <M extends Writable> Combiner<I, M> createCombiner() {
     Class<? extends Combiner<I, M>> klass = classes.getCombinerClass();
     return ReflectionUtils.newInstance(klass, this);
   }
@@ -409,41 +392,17 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return User's vertex resolver class
    */
-  public Class<? extends VertexResolver<I, V, E, M>> getVertexResolverClass() {
+  public Class<? extends VertexResolver<I, V, E>> getVertexResolverClass() {
     return classes.getVertexResolverClass();
   }
 
   /**
    * Create a user vertex revolver
    *
-   * @param graphState State of the graph from the worker
    * @return Instantiated user vertex resolver
    */
-  @SuppressWarnings("rawtypes")
-  public VertexResolver<I, V, E, M> createVertexResolver(
-                       GraphState<I, V, E, M> graphState) {
-    VertexResolver<I, V, E, M> resolver =
-        ReflectionUtils.newInstance(getVertexResolverClass(), this);
-    resolver.setGraphState(graphState);
-    return resolver;
-  }
-
-  /**
-   * Get the user's subclassed PartitionContext.
-   *
-   * @return User's partition context class
-   */
-  public Class<? extends PartitionContext> getPartitionContextClass() {
-    return classes.getPartitionContextClass();
-  }
-
-  /**
-   * Create a user partition context
-   *
-   * @return Instantiated user partition context
-   */
-  public PartitionContext createPartitionContext() {
-    return ReflectionUtils.newInstance(getPartitionContextClass(), this);
+  public VertexResolver<I, V, E> createVertexResolver() {
+    return ReflectionUtils.newInstance(getVertexResolverClass(), this);
   }
 
   /**
@@ -458,15 +417,10 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   /**
    * Create a user worker context
    *
-   * @param graphState State of the graph from the worker
    * @return Instantiated user worker context
    */
-  @SuppressWarnings("rawtypes")
-  public WorkerContext createWorkerContext(GraphState<I, V, E, M> graphState) {
-    WorkerContext workerContext =
-        ReflectionUtils.newInstance(getWorkerContextClass(), this);
-    workerContext.setGraphState(graphState);
-    return workerContext;
+  public WorkerContext createWorkerContext() {
+    return ReflectionUtils.newInstance(getWorkerContextClass(), this);
   }
 
   /**
@@ -488,17 +442,29 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   }
 
   @Override
-  public Class<? extends Vertex<I, V, E, M>> getVertexClass() {
-    return classes.getVertexClass();
+  public Class<? extends
+      Computation<I, V, E, ? extends Writable, ? extends Writable>>
+  getComputationClass() {
+    return classes.getComputationClass();
   }
 
   /**
-   * Create a user vertex
+   * Create a user computation
    *
-   * @return Instantiated user vertex
+   * @return Instantiated user computation
    */
-  public Vertex<I, V, E, M> createVertex() {
-    return ReflectionUtils.newInstance(getVertexClass(), this);
+  public Computation<I, V, E, ? extends Writable, ? extends Writable>
+  createComputation() {
+    return ReflectionUtils.newInstance(getComputationClass(), this);
+  }
+
+  /**
+   * Create a vertex
+   *
+   * @return Instantiated vertex
+   */
+  public Vertex<I, V, E> createVertex() {
+    return ReflectionUtils.newInstance(Vertex.class, this);
   }
 
   /**
@@ -658,34 +624,58 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   }
 
   /**
-   * Get the user's subclassed vertex message value class.
+   * Get the user's subclassed incoming message value class.
    *
+   * @param <M> Message data
    * @return User's vertex message value class
    */
-  @SuppressWarnings("unchecked")
-  public Class<M> getMessageValueClass() {
-    return classes.getMessageValueClass();
+  public <M extends Writable> Class<M> getIncomingMessageValueClass() {
+    return classes.getIncomingMessageValueClass();
   }
 
   /**
-   * Create a user vertex message value
+   * Get the user's subclassed outgoing message value class.
    *
-   * @return Instantiated user vertex message value
+   * @param <M> Message data
+   * @return User's vertex message value class
    */
-  public M createMessageValue() {
-    Class<M> klass = getMessageValueClass();
-    if (klass == NullWritable.class) {
+  public <M extends Writable> Class<M> getOutgoingMessageValueClass() {
+    return classes.getOutgoingMessageValueClass();
+  }
+
+  /**
+   * Create incoming message value
+   *
+   * @param <M> Message data
+   * @return Incoming message value
+   */
+  public <M extends Writable> M createIncomingMessageValue() {
+    return this.<M>createMessageValue(this.<M>getIncomingMessageValueClass());
+  }
+
+  /**
+   * Create outgoing message value
+   *
+   * @param <M> Message data
+   * @return Outgoing message value
+   */
+  public <M extends Writable> M createOutgoingMessageValue() {
+    return this.<M>createMessageValue(this.<M>getOutgoingMessageValueClass());
+  }
+
+  /**
+   * Create a message value
+   *
+   * @param <M> Message data
+   * @param messageClass Message class
+   * @return Instantiated message value
+   */
+  private <M extends Writable> M createMessageValue(
+      Class<? extends Writable> messageClass) {
+    if (messageClass == NullWritable.class) {
       return (M) NullWritable.get();
     } else {
-      try {
-        return klass.newInstance();
-      } catch (InstantiationException e) {
-        throw new IllegalArgumentException(
-            "createMessageValue: Failed to instantiate", e);
-      } catch (IllegalAccessException e) {
-        throw new IllegalArgumentException(
-            "createMessageValue: Illegally accessed", e);
-      }
+      return (M) ReflectionUtils.newInstance(messageClass);
     }
   }
 
@@ -792,10 +782,10 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    * @param progressable Progressable for reporting progress
    * @return Instantiated partition
    */
-  public Partition<I, V, E, M> createPartition(
+  public Partition<I, V, E> createPartition(
       int id, Progressable progressable) {
-    Class<? extends Partition<I, V, E, M>> klass = classes.getPartitionClass();
-    Partition<I, V, E, M> partition = ReflectionUtils.newInstance(klass, this);
+    Class<? extends Partition<I, V, E>> klass = classes.getPartitionClass();
+    Partition<I, V, E> partition = ReflectionUtils.newInstance(klass, this);
     partition.initialize(id, progressable);
     return partition;
   }
@@ -868,4 +858,14 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
       return new ExtendedByteArrayDataInput(buf, off, length);
     }
   }
+
+  /**
+   * Update Computation and Combiner class used
+   *
+   * @param superstepClasses SuperstepClasses
+   */
+  public void updateSuperstepClasses(SuperstepClasses superstepClasses) {
+    classes.setComputationClass(superstepClasses.getComputationClass());
+    classes.setCombinerClass(superstepClasses.getCombinerClass());
+  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
index 631b209..a983ee4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
@@ -31,6 +31,6 @@ import org.apache.hadoop.io.WritableComparable;
  */
 public abstract class ConfigurableOutEdges<I extends WritableComparable,
     E extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E,
-    Writable> implements OutEdges<I, E> {
+    extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E>
+    implements OutEdges<I, E> {
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
index 9e2d246..420bf93 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
@@ -41,16 +41,15 @@ import org.apache.log4j.Logger;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class EdgeStore<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(EdgeStore.class);
   /** Service worker. */
-  private CentralizedServiceWorker<I, V, E, M> service;
+  private CentralizedServiceWorker<I, V, E> service;
   /** Giraph configuration. */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+  private ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /** Progressable to report progress. */
   private Progressable progressable;
   /** Map used to temporarily store incoming edges. */
@@ -75,8 +74,8 @@ public class EdgeStore<I extends WritableComparable,
    * @param progressable Progressable
    */
   public EdgeStore(
-      CentralizedServiceWorker<I, V, E, M> service,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
+      CentralizedServiceWorker<I, V, E> service,
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
       Progressable progressable) {
     this.service = service;
     this.configuration = configuration;
@@ -179,14 +178,14 @@ public class EdgeStore<I extends WritableComparable,
           public Void call() throws Exception {
             Integer partitionId;
             while ((partitionId = partitionIdQueue.poll()) != null) {
-              Partition<I, V, E, M> partition =
+              Partition<I, V, E> partition =
                   service.getPartitionStore().getPartition(partitionId);
               ConcurrentMap<I, OutEdges<I, E>> partitionEdges =
                   transientEdges.remove(partitionId);
               for (I vertexId : partitionEdges.keySet()) {
                 OutEdges<I, E> outEdges = convertInputToComputeEdges(
                     partitionEdges.remove(vertexId));
-                Vertex<I, V, E, M> vertex = partition.getVertex(vertexId);
+                Vertex<I, V, E> vertex = partition.getVertex(vertexId);
                 // If the source vertex doesn't exist, create it. Otherwise,
                 // just set the edges.
                 if (vertex == null) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
index 82486f4..ca32a9f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
@@ -34,14 +34,14 @@ import java.util.Iterator;
 public class MutableEdgesIterable<I extends WritableComparable,
     E extends Writable> implements Iterable<MutableEdge<I, E>> {
   /** Vertex that owns the out-edges. */
-  private Vertex<I, ?, E, ?> vertex;
+  private Vertex<I, ?, E> vertex;
 
   /**
    * Constructor.
    *
    * @param vertex Owning vertex
    */
-  public MutableEdgesIterable(Vertex<I, ?, E, ?> vertex) {
+  public MutableEdgesIterable(Vertex<I, ?, E> vertex) {
     this.vertex = vertex;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
index cd845d0..529234d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
@@ -73,7 +73,7 @@ public class MutableEdgesWrapper<I extends WritableComparable,
   public static <I extends WritableComparable, E extends Writable>
   MutableEdgesWrapper<I, E> wrap(
       OutEdges<I, E> edges,
-      ImmutableClassesGiraphConfiguration<I, ?, E, ?> conf) {
+      ImmutableClassesGiraphConfiguration<I, ?, E> conf) {
     MutableEdgesWrapper<I, E> wrapper = new MutableEdgesWrapper<I, E>(
         edges, conf.createAndInitializeOutEdges(edges.size()));
     return wrapper;

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/BasicComputation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/BasicComputation.java b/giraph-core/src/main/java/org/apache/giraph/graph/BasicComputation.java
new file mode 100644
index 0000000..180c5d3
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/BasicComputation.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.graph;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Computation in which both incoming and outgoing message types are the same.
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ * @param <M> Message type
+ */
+public abstract class BasicComputation<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable>
+    extends Computation<I, V, E, M, M> {
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java b/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java
new file mode 100644
index 0000000..84158df
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/Computation.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.graph;
+
+import org.apache.giraph.comm.WorkerClientRequestProcessor;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import java.io.IOException;
+
+/**
+ * Basic abstract class for writing a BSP application for computation.
+ *
+ * During the superstep there can be several instances of this class,
+ * each doing computation on one partition of the graph's vertices.
+ *
+ * Note that each thread will have its own {@link Computation},
+ * so accessing any data from this class is thread-safe.
+ * However, accessing global data (like data from {@link WorkerContext})
+ * is not thread-safe.
+ *
+ * Objects of this class only live for a single superstep.
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ * @param <M1> Incoming message type
+ * @param <M2> Outgoing message type
+ */
+public abstract class Computation<I extends WritableComparable,
+    V extends Writable, E extends Writable, M1 extends Writable,
+    M2 extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
+    implements WorkerAggregatorUsage {
+  /** Global graph state **/
+  private GraphState graphState;
+  /** Handles requests */
+  private WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor;
+  /** Graph-wide BSP Mapper for this Computation */
+  private GraphTaskManager<I, V, E> graphTaskManager;
+  /** Worker aggregator usage */
+  private WorkerAggregatorUsage workerAggregatorUsage;
+  /** Worker context */
+  private WorkerContext workerContext;
+
+  /**
+   * Must be defined by user to do computation on a single Vertex.
+   *
+   * @param vertex   Vertex
+   * @param messages Messages that were sent to this vertex in the previous
+   *                 superstep.  Each message is only guaranteed to have
+   *                 a life expectancy as long as next() is not called.
+   */
+  public abstract void compute(Vertex<I, V, E> vertex,
+      Iterable<M1> messages) throws IOException;
+
+  /**
+   * Prepare for computation. This method is executed exactly once prior to
+   * {@link #compute(Vertex, Iterable)} being called for any of the vertices
+   * in the partition.
+   */
+  public void preSuperstep() {
+  }
+
+  /**
+   * Finish computation. This method is executed exactly once after computation
+   * for all vertices in the partition is complete.
+   */
+  public void postSuperstep() {
+  }
+
+  /**
+   * Initialize, called by infrastructure before the superstep starts.
+   * Shouldn't be called by user code.
+   *
+   * @param graphState Graph state
+   * @param workerClientRequestProcessor Processor for handling requests
+   * @param graphTaskManager Graph-wide BSP Mapper for this Vertex
+   * @param workerAggregatorUsage Worker aggregator usage
+   * @param workerContext Worker context
+   */
+  public final void initialize(
+      GraphState graphState,
+      WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor,
+      GraphTaskManager<I, V, E> graphTaskManager,
+      WorkerAggregatorUsage workerAggregatorUsage,
+      WorkerContext workerContext) {
+    this.graphState = graphState;
+    this.workerClientRequestProcessor = workerClientRequestProcessor;
+    this.graphTaskManager = graphTaskManager;
+    this.workerAggregatorUsage = workerAggregatorUsage;
+    this.workerContext = workerContext;
+  }
+
+  /**
+   * Retrieves the current superstep.
+   *
+   * @return Current superstep
+   */
+  public long getSuperstep() {
+    return graphState.getSuperstep();
+  }
+
+  /**
+   * Get the total (all workers) number of vertices that
+   * existed in the previous superstep.
+   *
+   * @return Total number of vertices (-1 if first superstep)
+   */
+  public long getTotalNumVertices() {
+    return graphState.getTotalNumVertices();
+  }
+
+  /**
+   * Get the total (all workers) number of edges that
+   * existed in the previous superstep.
+   *
+   * @return Total number of edges (-1 if first superstep)
+   */
+  public long getTotalNumEdges() {
+    return graphState.getTotalNumEdges();
+  }
+
+  /**
+   * Send a message to a vertex id.
+   *
+   * @param id Vertex id to send the message to
+   * @param message Message data to send
+   */
+  public void sendMessage(I id, M2 message) {
+    if (workerClientRequestProcessor.sendMessageRequest(id, message)) {
+      graphTaskManager.notifySentMessages();
+    }
+  }
+
+  /**
+   * Send a message to all edges.
+   *
+   * @param vertex Vertex whose edges to send the message to.
+   * @param message Message sent to all edges.
+   */
+  public void sendMessageToAllEdges(Vertex<I, V, E> vertex, M2 message) {
+    for (Edge<I, E> edge : vertex.getEdges()) {
+      sendMessage(edge.getTargetVertexId(), message);
+    }
+  }
+
+  /**
+   * Sends a request to create a vertex that will be available during the
+   * next superstep.
+   *
+   * @param id Vertex id
+   * @param value Vertex value
+   * @param edges Initial edges
+   */
+  public void addVertexRequest(I id, V value,
+      OutEdges<I, E> edges) throws IOException {
+    Vertex<I, V, E> vertex = getConf().createVertex();
+    vertex.initialize(id, value, edges);
+    workerClientRequestProcessor.addVertexRequest(vertex);
+  }
+
+  /**
+   * Sends a request to create a vertex that will be available during the
+   * next superstep.
+   *
+   * @param id Vertex id
+   * @param value Vertex value
+   */
+  public void addVertexRequest(I id, V value) throws IOException {
+    addVertexRequest(id, value, getConf().createAndInitializeOutEdges());
+  }
+
+  /**
+   * Request to remove a vertex from the graph
+   * (applied just prior to the next superstep).
+   *
+   * @param vertexId Id of the vertex to be removed.
+   */
+  public void removeVertexRequest(I vertexId) throws IOException {
+    workerClientRequestProcessor.removeVertexRequest(vertexId);
+  }
+
+  /**
+   * Request to add an edge of a vertex in the graph
+   * (processed just prior to the next superstep)
+   *
+   * @param sourceVertexId Source vertex id of edge
+   * @param edge Edge to add
+   */
+  public void addEdgeRequest(I sourceVertexId,
+      Edge<I, E> edge) throws IOException {
+    workerClientRequestProcessor.addEdgeRequest(sourceVertexId, edge);
+  }
+
+  /**
+   * Request to remove all edges from a given source vertex to a given target
+   * vertex (processed just prior to the next superstep).
+   *
+   * @param sourceVertexId Source vertex id
+   * @param targetVertexId Target vertex id
+   */
+  public void removeEdgesRequest(I sourceVertexId,
+      I targetVertexId) throws IOException {
+    workerClientRequestProcessor.removeEdgesRequest(
+        sourceVertexId, targetVertexId);
+  }
+
+  /**
+   * Get the mapper context
+   *
+   * @return Mapper context
+   */
+  public Mapper.Context getContext() {
+    return graphState.getContext();
+  }
+
+  /**
+   * Get the worker context
+   *
+   * @param <W> WorkerContext class
+   * @return WorkerContext context
+   */
+  @SuppressWarnings("unchecked")
+  public <W extends WorkerContext> W getWorkerContext() {
+    return (W) workerContext;
+  }
+
+  @Override
+  public <A extends Writable> void aggregate(String name, A value) {
+    workerAggregatorUsage.aggregate(name, value);
+  }
+
+  @Override
+  public <A extends Writable> A getAggregatedValue(String name) {
+    return workerAggregatorUsage.<A>getAggregatedValue(name);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java b/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
index 0fc5fdf..6fdcfb0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/ComputeCallable.java
@@ -28,7 +28,6 @@ import org.apache.giraph.metrics.MetricNames;
 import org.apache.giraph.metrics.SuperstepMetricsRegistry;
 import org.apache.giraph.metrics.TimerDesc;
 import org.apache.giraph.partition.Partition;
-import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.partition.PartitionStats;
 import org.apache.giraph.time.SystemTime;
 import org.apache.giraph.time.Time;
@@ -65,10 +64,11 @@ import java.util.concurrent.Callable;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
+ * @param <M1> Incoming message type
+ * @param <M2> Outgoing message type
  */
 public class ComputeCallable<I extends WritableComparable, V extends Writable,
-    E extends Writable, M extends Writable>
+    E extends Writable, M1 extends Writable, M2 extends Writable>
     implements Callable<Collection<PartitionStats>> {
   /** Class logger */
   private static final Logger LOG  = Logger.getLogger(ComputeCallable.class);
@@ -76,21 +76,18 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
   private static final Time TIME = SystemTime.get();
   /** Context */
   private final Mapper<?, ?, ?, ?>.Context context;
-  /** Graph state (note that it is recreated in call() for locality) */
-  private GraphState<I, V, E, M> graphState;
+  /** Graph state */
+  private final GraphState graphState;
   /** Thread-safe queue of all partition ids */
   private final BlockingQueue<Integer> partitionIdQueue;
   /** Message store */
-  private final MessageStoreByPartition<I, M> messageStore;
+  private final MessageStoreByPartition<I, M1> messageStore;
   /** Configuration */
-  private final ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+  private final ImmutableClassesGiraphConfiguration<I, V, E> configuration;
   /** Worker (for NettyWorkerClientRequestProcessor) */
-  private final CentralizedServiceWorker<I, V, E, M> serviceWorker;
+  private final CentralizedServiceWorker<I, V, E> serviceWorker;
   /** Dump some progress every 30 seconds */
   private final TimedLogger timedLogger = new TimedLogger(30 * 1000, LOG);
-  /** Sends the messages (unique per Callable) */
-  private WorkerClientRequestProcessor<I, V, E, M>
-  workerClientRequestProcessor;
   /** VertexWriter for this ComputeCallable */
   private SimpleVertexWriter<I, V, E> vertexWriter;
   /** Get the start time in nanos */
@@ -113,17 +110,16 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
    * @param serviceWorker Service worker
    */
   public ComputeCallable(
-      Mapper<?, ?, ?, ?>.Context context, GraphState<I, V, E, M> graphState,
-      MessageStoreByPartition<I, M> messageStore,
+      Mapper<?, ?, ?, ?>.Context context, GraphState graphState,
+      MessageStoreByPartition<I, M1> messageStore,
       BlockingQueue<Integer> partitionIdQueue,
-      ImmutableClassesGiraphConfiguration<I, V, E, M> configuration,
-      CentralizedServiceWorker<I, V, E, M> serviceWorker) {
+      ImmutableClassesGiraphConfiguration<I, V, E> configuration,
+      CentralizedServiceWorker<I, V, E> serviceWorker) {
     this.context = context;
     this.configuration = configuration;
     this.partitionIdQueue = partitionIdQueue;
     this.messageStore = messageStore;
     this.serviceWorker = serviceWorker;
-    // Will be replaced later in call() for locality
     this.graphState = graphState;
 
     SuperstepMetricsRegistry metrics = GiraphMetrics.get().perSuperstep();
@@ -136,16 +132,12 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
   @Override
   public Collection<PartitionStats> call() {
     // Thread initialization (for locality)
-    this.workerClientRequestProcessor =
-        new NettyWorkerClientRequestProcessor<I, V, E, M>(
+    WorkerClientRequestProcessor<I, V, E> workerClientRequestProcessor =
+        new NettyWorkerClientRequestProcessor<I, V, E>(
             context, configuration, serviceWorker);
     WorkerThreadAggregatorUsage aggregatorUsage =
         serviceWorker.getAggregatorHandler().newThreadAggregatorUsage();
-
-    this.graphState = new GraphState<I, V, E, M>(graphState.getSuperstep(),
-        graphState.getTotalNumVertices(), graphState.getTotalNumEdges(),
-        context, graphState.getGraphTaskManager(), workerClientRequestProcessor,
-        aggregatorUsage);
+    WorkerContext workerContext = serviceWorker.getWorkerContext();
 
     vertexWriter = serviceWorker.getSuperstepOutput().getVertexWriter();
 
@@ -156,10 +148,18 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
         break;
       }
 
-      Partition<I, V, E, M> partition =
+      Partition<I, V, E> partition =
           serviceWorker.getPartitionStore().getPartition(partitionId);
+
+      Computation<I, V, E, M1, M2> computation =
+          (Computation<I, V, E, M1, M2>) configuration.createComputation();
+      computation.initialize(graphState, workerClientRequestProcessor,
+          serviceWorker.getGraphTaskManager(), aggregatorUsage, workerContext);
+      computation.preSuperstep();
+
       try {
-        PartitionStats partitionStats = computePartition(partition);
+        PartitionStats partitionStats =
+            computePartition(computation, partition);
         partitionStatsList.add(partitionStats);
         long partitionMsgs = workerClientRequestProcessor.resetMessageCount();
         partitionStats.addMessagesSentCount(partitionMsgs);
@@ -177,6 +177,8 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
       } finally {
         serviceWorker.getPartitionStore().putPartition(partition);
       }
+
+      computation.postSuperstep();
     }
 
     // Return VertexWriter after the usage
@@ -201,29 +203,19 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
   /**
    * Compute a single partition
    *
+   * @param computation Computation to use
    * @param partition Partition to compute
    * @return Partition stats for this computed partition
    */
-  private PartitionStats computePartition(Partition<I, V, E, M> partition)
-    throws IOException, InterruptedException {
+  private PartitionStats computePartition(
+      Computation<I, V, E, M1, M2> computation,
+      Partition<I, V, E> partition) throws IOException, InterruptedException {
     PartitionStats partitionStats =
         new PartitionStats(partition.getId(), 0, 0, 0, 0);
     // Make sure this is thread-safe across runs
     synchronized (partition) {
-      // Prepare Partition context
-      WorkerContext workerContext =
-          graphState.getGraphTaskManager().getWorkerContext();
-      PartitionContext partitionContext = partition.getPartitionContext();
-      synchronized (workerContext) {
-        partitionContext.preSuperstep(workerContext);
-      }
-      graphState.setPartitionContext(partition.getPartitionContext());
-
-      for (Vertex<I, V, E, M> vertex : partition) {
-        // Make sure every vertex has this thread's
-        // graphState before computing
-        vertex.setGraphState(graphState);
-        Iterable<M> messages = messageStore.getVertexMessages(vertex.getId());
+      for (Vertex<I, V, E> vertex : partition) {
+        Iterable<M1> messages = messageStore.getVertexMessages(vertex.getId());
         if (vertex.isHalted() && !Iterables.isEmpty(messages)) {
           vertex.wakeUp();
         }
@@ -231,7 +223,7 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
           context.progress();
           TimerContext computeOneTimerContext = computeOneTimer.time();
           try {
-            vertex.compute(messages);
+            computation.compute(vertex, messages);
           } finally {
             computeOneTimerContext.stop();
           }
@@ -254,10 +246,6 @@ public class ComputeCallable<I extends WritableComparable, V extends Writable,
       }
 
       messageStore.clearPartition(partition.getId());
-
-      synchronized (workerContext) {
-        partitionContext.postSuperstep(workerContext);
-      }
     }
     return partitionStats;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexResolver.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexResolver.java b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexResolver.java
index 52df38d..748c3a6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexResolver.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexResolver.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.graph;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
 import org.apache.hadoop.io.Writable;
@@ -32,29 +32,24 @@ import org.apache.log4j.Logger;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class DefaultVertexResolver<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements VertexResolver<I, V, E, M>,
-    ImmutableClassesGiraphConfigurable<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
+    implements VertexResolver<I, V, E> {
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(
       DefaultVertexResolver.class);
-  /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf = null;
-  /** Stored graph state */
-  private GraphState<I, V, E, M> graphState;
 
   /** Whether to create vertices when they receive a message */
   private boolean createVertexesOnMessages = true;
 
   @Override
-  public Vertex<I, V, E, M> resolve(
+  public Vertex<I, V, E> resolve(
       I vertexId,
-      Vertex<I, V, E, M> vertex,
-      VertexChanges<I, V, E, M> vertexChanges,
+      Vertex<I, V, E> vertex,
+      VertexChanges<I, V, E> vertexChanges,
       boolean hasMessages) {
     // This is the default vertex resolution algorithm
 
@@ -80,8 +75,8 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param vertex Vertex to remove edges from
    * @param vertexChanges contains list of edges to remove.
    */
-  protected void removeEdges(Vertex<I, V, E, M> vertex,
-                             VertexChanges<I, V, E, M> vertexChanges) {
+  protected void removeEdges(Vertex<I, V, E> vertex,
+                             VertexChanges<I, V, E> vertexChanges) {
     if (vertex == null) {
       return;
     }
@@ -101,9 +96,9 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param vertexChanges specifies if we should remove vertex
    * @return null if vertex should be removed, otherwise the vertex itself.
    */
-  protected Vertex<I, V, E, M> removeVertexIfDesired(
-      Vertex<I, V, E, M> vertex,
-      VertexChanges<I, V, E, M> vertexChanges) {
+  protected Vertex<I, V, E> removeVertexIfDesired(
+      Vertex<I, V, E> vertex,
+      VertexChanges<I, V, E> vertexChanges) {
     if (hasVertexRemovals(vertexChanges)) {
       vertex = null;
     }
@@ -120,18 +115,17 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param hasMessages true if this vertex received any messages
    * @return Vertex created or passed in, or null if no vertex should be added
    */
-  protected Vertex<I, V, E, M> addVertexIfDesired(
+  protected Vertex<I, V, E> addVertexIfDesired(
       I vertexId,
-      Vertex<I, V, E, M> vertex,
-      VertexChanges<I, V, E, M> vertexChanges,
+      Vertex<I, V, E> vertex,
+      VertexChanges<I, V, E> vertexChanges,
       boolean hasMessages) {
     if (vertex == null) {
       if (hasVertexAdditions(vertexChanges)) {
         vertex = vertexChanges.getAddedVertexList().get(0);
       } else if ((hasMessages && createVertexesOnMessages) ||
                  hasEdgeAdditions(vertexChanges)) {
-        vertex = conf.createVertex();
-        vertex.setGraphState(graphState);
+        vertex = getConf().createVertex();
         vertex.initialize(vertexId, getConf().createVertexValue());
       }
     } else if (hasVertexAdditions(vertexChanges)) {
@@ -148,8 +142,8 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param vertex Vertex to add edges to
    * @param vertexChanges contains edges to add
    */
-  protected void addEdges(Vertex<I, V, E, M> vertex,
-                          VertexChanges<I, V, E, M> vertexChanges) {
+  protected void addEdges(Vertex<I, V, E> vertex,
+                          VertexChanges<I, V, E> vertexChanges) {
     if (vertex == null) {
       return;
     }
@@ -166,7 +160,7 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param changes VertexChanges to check
    * @return true if changes contains vertex removal requests
    */
-  protected  boolean hasVertexRemovals(VertexChanges<I, V, E, M> changes) {
+  protected  boolean hasVertexRemovals(VertexChanges<I, V, E> changes) {
     return changes != null && changes.getRemovedVertexCount() > 0;
   }
 
@@ -176,7 +170,7 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param changes VertexChanges to check
    * @return true if changes contains vertex addition requests
    */
-  protected boolean hasVertexAdditions(VertexChanges<I, V, E, M> changes) {
+  protected boolean hasVertexAdditions(VertexChanges<I, V, E> changes) {
     return changes != null && !changes.getAddedVertexList().isEmpty();
   }
 
@@ -186,7 +180,7 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param changes VertexChanges to check
    * @return true if changes contains edge addition requests
    */
-  protected boolean hasEdgeAdditions(VertexChanges<I, V, E, M> changes) {
+  protected boolean hasEdgeAdditions(VertexChanges<I, V, E> changes) {
     return changes != null && !changes.getAddedEdgeList().isEmpty();
   }
 
@@ -196,28 +190,13 @@ public class DefaultVertexResolver<I extends WritableComparable,
    * @param changes VertexChanges to check
    * @return true if changes contains edge removal requests
    */
-  protected boolean hasEdgeRemovals(VertexChanges<I, V, E, M> changes) {
+  protected boolean hasEdgeRemovals(VertexChanges<I, V, E> changes) {
     return changes != null && !changes.getRemovedEdgeList().isEmpty();
   }
 
   @Override
-  public ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(ImmutableClassesGiraphConfiguration<I, V, E, M> conf) {
-    this.conf = conf;
+  public void setConf(ImmutableClassesGiraphConfiguration<I, V, E> conf) {
+    super.setConf(conf);
     createVertexesOnMessages = conf.getResolverCreateVertexOnMessages();
   }
-
-  @Override
-  public GraphState<I, V, E, M> getGraphState() {
-    return graphState;
-  }
-
-  @Override
-  public void setGraphState(GraphState<I, V, E, M> graphState) {
-    this.graphState = graphState;
-  }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java
index 47902d1..adbe9d3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java
@@ -35,7 +35,7 @@ public class DefaultVertexValueFactory<V extends Writable>
 
   @Override
   public void initialize(
-      ImmutableClassesGiraphConfiguration<?, V, ?, ?> configuration) {
+      ImmutableClassesGiraphConfiguration<?, V, ?> configuration) {
     vertexValueClass = configuration.getVertexValueClass();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/GiraphTransferRegulator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GiraphTransferRegulator.java b/giraph-core/src/main/java/org/apache/giraph/graph/GiraphTransferRegulator.java
index 3c2286d..19034ca 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GiraphTransferRegulator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/GiraphTransferRegulator.java
@@ -155,12 +155,11 @@ public class GiraphTransferRegulator {
    * @param <I> the vertex id type.
    * @param <V> the vertex value type.
    * @param <E> the edge value type.
-   * @param <M> the message value type.
    */
   public <I extends WritableComparable, V extends Writable,
-  E extends Writable, M extends Writable> void
+  E extends Writable> void
   incrementCounters(PartitionOwner partitionOwner,
-    Vertex<I, V, E, M> vertex) {
+    Vertex<I, V, E> vertex) {
     final int id = partitionOwner.getPartitionId();
     // vertex counts
     vertexAccumulator

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java b/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
index 4181d24..c86a024 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/GraphMapper.java
@@ -36,16 +36,15 @@ import java.io.IOException;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class GraphMapper<I extends WritableComparable, V extends Writable,
-    E extends Writable, M extends Writable> extends
+    E extends Writable> extends
     Mapper<Object, Object, Object, Object> {
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(GraphMapper.class);
   /** Manage the framework-agnostic Giraph tasks for this job run */
-  private GraphTaskManager<I, V, E, M> graphTaskManager;
+  private GraphTaskManager<I, V, E> graphTaskManager;
 
   @Override
   public void setup(Context context)
@@ -56,7 +55,7 @@ public class GraphMapper<I extends WritableComparable, V extends Writable,
 
     // Execute all Giraph-related role(s) assigned to this compute node.
     // Roles can include "master," "worker," "zookeeper," or . . . ?
-    graphTaskManager = new GraphTaskManager<I, V, E, M>(context);
+    graphTaskManager = new GraphTaskManager<I, V, E>(context);
     graphTaskManager.setup(
       DistributedCache.getLocalCacheArchives(context.getConfiguration()));
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/GraphState.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GraphState.java b/giraph-core/src/main/java/org/apache/giraph/graph/GraphState.java
index 93ad5df..ca57008 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GraphState.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/GraphState.java
@@ -15,26 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.giraph.graph;
 
-import org.apache.giraph.comm.WorkerClientRequestProcessor;
-import org.apache.giraph.partition.PartitionContext;
-import org.apache.giraph.worker.WorkerAggregatorUsage;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 
 /**
  * Immutable global state of the graph.
- *
- * @param <I> Vertex id
- * @param <V> Vertex data
- * @param <E> Edge data
- * @param <M> Message data
  */
-@SuppressWarnings("rawtypes")
-public class GraphState<I extends WritableComparable, V extends Writable,
-E extends Writable, M extends Writable> {
+public class GraphState {
   /** Graph-wide superstep */
   private final long superstep;
   /** Graph-wide number of vertices */
@@ -43,15 +32,6 @@ E extends Writable, M extends Writable> {
   private final long numEdges;
   /** Graph-wide map context */
   private final Mapper<?, ?, ?, ?>.Context context;
-  /** Graph-wide BSP Mapper for this Vertex */
-  private final GraphTaskManager<I, V, E, M> graphTaskManager;
-  /** Handles requests */
-  private final WorkerClientRequestProcessor<I, V, E, M>
-  workerClientRequestProcessor;
-  /** Worker aggregator usage */
-  private final WorkerAggregatorUsage workerAggregatorUsage;
-  /** Partition context */
-  private PartitionContext partitionContext;
 
   /**
    * Constructor
@@ -60,24 +40,14 @@ E extends Writable, M extends Writable> {
    * @param numVertices Current graph-wide vertices
    * @param numEdges Current graph-wide edges
    * @param context Context
-   * @param graphTaskManager GraphTaskManager for this compute node
-   * @param workerClientRequestProcessor Handles all communication
-   * @param workerAggregatorUsage Aggregator usage
    *
    */
-  public GraphState(
-      long superstep, long numVertices,
-      long numEdges, Mapper<?, ?, ?, ?>.Context context,
-      GraphTaskManager<I, V, E, M> graphTaskManager,
-      WorkerClientRequestProcessor<I, V, E, M> workerClientRequestProcessor,
-      WorkerAggregatorUsage workerAggregatorUsage) {
+  public GraphState(long superstep, long numVertices, long numEdges,
+      Mapper<?, ?, ?, ?>.Context context) {
     this.superstep = superstep;
     this.numVertices = numVertices;
     this.numEdges = numEdges;
     this.context = context;
-    this.graphTaskManager = graphTaskManager;
-    this.workerClientRequestProcessor = workerClientRequestProcessor;
-    this.workerAggregatorUsage = workerAggregatorUsage;
   }
 
   public long getSuperstep() {
@@ -96,33 +66,9 @@ E extends Writable, M extends Writable> {
     return context;
   }
 
-  public GraphTaskManager<I, V, E, M> getGraphTaskManager() {
-    return graphTaskManager;
-  }
-
-  public WorkerClientRequestProcessor<I, V, E, M>
-  getWorkerClientRequestProcessor() {
-    return workerClientRequestProcessor;
-  }
-
-  public WorkerAggregatorUsage getWorkerAggregatorUsage() {
-    return workerAggregatorUsage;
-  }
-
-  public void setPartitionContext(PartitionContext partitionContext) {
-    this.partitionContext = partitionContext;
-  }
-
-  public PartitionContext getPartitionContext() {
-    return partitionContext;
-  }
-
   @Override
   public String toString() {
     return "(superstep=" + superstep + ",numVertices=" + numVertices + "," +
-        "numEdges=" + numEdges + ",context=" + context +
-        ",graphMapper=" + graphTaskManager +
-        ",workerClientRequestProcessor=" + workerClientRequestProcessor + ")";
-
+        "numEdges=" + numEdges + ",context=" + context + ")";
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/8811165e/giraph-core/src/main/java/org/apache/giraph/graph/GraphStateAware.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/GraphStateAware.java b/giraph-core/src/main/java/org/apache/giraph/graph/GraphStateAware.java
deleted file mode 100644
index 76cef43..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/graph/GraphStateAware.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.graph;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Interface specifying that the class can be configured with a GraphState.
- *
- * @param <I> Vertex ID object
- * @param <V> Vertex Value object
- * @param <E> Edge object
- * @param <M> Message object
- */
-public interface GraphStateAware<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
-  /**
-   * Set the graph state.
-   *
-   * @param graphState Graph state saved.
-   */
-  void setGraphState(GraphState<I, V, E, M> graphState);
-
-  /**
-   * Get the graph state stored.
-   *
-   * @return GraphState stored.
-   */
-  GraphState<I, V, E, M> getGraphState();
-}