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/04/03 21:13:28 UTC

[1/2] GIRAPH-589: Remove unnecessary generics from input formats (majakabiljo)

Updated Branches:
  refs/heads/trunk ab88282d1 -> c5c1c4c61


http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
new file mode 100644
index 0000000..fdc9050
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
@@ -0,0 +1,106 @@
+/*
+ * 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.conf.ImmutableClassesGiraphConfigurable;
+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.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;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+/**
+ * Input format for unweighted graphs with long ids and double vertex values
+ */
+public class LongDoubleNullTextInputFormat
+    extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable>
+    implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
+    NullWritable, Writable> {
+  /** Configuration. */
+  private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
+      NullWritable, Writable> conf;
+
+  @Override
+  public TextVertexReader createVertexReader(InputSplit split,
+                                             TaskAttemptContext context)
+    throws IOException {
+    return new LongDoubleNullDoubleVertexReader();
+  }
+
+  @Override
+  public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
+      DoubleWritable, NullWritable, Writable> configuration) {
+    this.conf = configuration;
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
+      NullWritable, Writable> getConf() {
+    return conf;
+  }
+
+  /**
+   * Vertex reader associated with
+   * {@link LongDoubleNullTextInputFormat}.
+   */
+  public class LongDoubleNullDoubleVertexReader extends
+      TextVertexInputFormat<LongWritable, DoubleWritable,
+          NullWritable>.TextVertexReader {
+    /** Separator of the vertex and neighbors */
+    private final Pattern separator = Pattern.compile("[\t ]");
+
+    @Override
+    public Vertex<LongWritable, DoubleWritable, NullWritable, ?>
+    getCurrentVertex() throws IOException, InterruptedException {
+      Vertex<LongWritable, DoubleWritable, NullWritable, ?>
+          vertex = conf.createVertex();
+
+      String[] tokens =
+          separator.split(getRecordReader().getCurrentValue().toString());
+      List<Edge<LongWritable, NullWritable>> edges =
+          Lists.newArrayListWithCapacity(tokens.length - 1);
+      for (int n = 1; n < tokens.length; n++) {
+        edges.add(EdgeFactory.create(
+            new LongWritable(Long.parseLong(tokens[n])),
+            NullWritable.get()));
+      }
+
+      LongWritable vertexId = new LongWritable(Long.parseLong(tokens[0]));
+      vertex.initialize(vertexId, new DoubleWritable(), edges);
+
+      return vertex;
+    }
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+      return getRecordReader().nextKeyValue();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleDoubleTextInputFormat.java
deleted file mode 100644
index 50d527c..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleDoubleTextInputFormat.java
+++ /dev/null
@@ -1,139 +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.conf.ImmutableClassesGiraphConfigurable;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.io.formats.TextVertexInputFormat;
-import org.apache.giraph.edge.Edge;
-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.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.List;
-import java.util.regex.Pattern;
-
-/**
- * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
- * unweighted graphs with long ids. Each line consists of: vertex
- * neighbor1:weight1 neighbor2:weight2 ...
- */
-public class NormalizingLongDoubleDoubleDoubleTextInputFormat
-    extends
-    TextVertexInputFormat<LongWritable, DoubleWritable,
-      DoubleWritable, DoubleWritable>
-    implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
-    DoubleWritable, DoubleWritable> {
-  /** Configuration. */
-  private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, DoubleWritable> conf;
-
-  @Override
-  public TextVertexReader createVertexReader(
-      InputSplit split, TaskAttemptContext context) throws IOException {
-    return new NormalizingLongDoubleDoubleDoubleVertexReader();
-  }
-
-  @Override
-  public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
-      DoubleWritable, DoubleWritable, DoubleWritable> configuration) {
-    conf = configuration;
-  }
-
-  @Override
-  public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, DoubleWritable> getConf() {
-    return conf;
-  }
-
-  /**
-   * Vertex reader associated with
-   * {@link LongDoubleDoubleDoubleTextInputFormat}.
-   */
-  public class NormalizingLongDoubleDoubleDoubleVertexReader
-      extends TextVertexInputFormat.TextVertexReader {
-    /** Separator of the vertex and neighbors */
-    private final Pattern edgeSeparator = Pattern.compile("\\s+");
-    /** Separator of the edge id and edge weight */
-    private final Pattern weightSeparator = Pattern.compile(":");
-
-    @Override
-    public Vertex<LongWritable, DoubleWritable,
-    DoubleWritable, DoubleWritable> getCurrentVertex()
-      throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable,
-      DoubleWritable, DoubleWritable> vertex = conf.createVertex();
-
-      String[] tokens = edgeSeparator.split(getRecordReader()
-          .getCurrentValue().toString());
-      List<Edge<LongWritable, DoubleWritable>> edges = Lists
-          .newArrayListWithCapacity(tokens.length - 1);
-      parse(tokens, edges);
-      normalize(edges);
-
-      LongWritable vertexId = new LongWritable(Long.parseLong(tokens[0]));
-      vertex.initialize(vertexId, new DoubleWritable(), edges);
-
-      return vertex;
-    }
-
-    /**
-     * Parse a set of tokens into a map ID -> weight.
-     * @param tokens The tokens to be parsed.
-     * @param edges The map that will contain the result of the parsing.
-     */
-    void parse(String[] tokens,
-               Collection<Edge<LongWritable, DoubleWritable>> edges) {
-      for (int n = 1; n < tokens.length; n++) {
-        String[] parts = weightSeparator.split(tokens[n]);
-        edges.add(EdgeFactory.create(new LongWritable(Long.parseLong(parts[0])),
-            new DoubleWritable(Double.parseDouble(parts[1]))));
-      }
-    }
-
-    /**
-     * Normalize the edges with L1 normalization.
-     * @param edges The edges to be normalized.
-     */
-    void normalize(Collection<Edge<LongWritable, DoubleWritable>> edges) {
-      if (edges == null || edges.size() == 0) {
-        throw new IllegalArgumentException(
-            "Cannot normalize an empy set of edges");
-      }
-      float normalizer = 0.0f;
-      for (Edge<LongWritable, DoubleWritable> edge : edges) {
-        normalizer += edge.getValue().get();
-      }
-      for (Edge<LongWritable, DoubleWritable> edge : edges) {
-        edge.getValue().set(edge.getValue().get() / normalizer);
-      }
-    }
-
-    @Override
-    public boolean nextVertex() throws IOException, InterruptedException {
-      return getRecordReader().nextKeyValue();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
new file mode 100644
index 0000000..7dc8475
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
@@ -0,0 +1,139 @@
+/*
+ * 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.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.giraph.edge.Edge;
+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;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.regex.Pattern;
+
+/**
+ * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
+ * unweighted graphs with long ids. Each line consists of: vertex
+ * neighbor1:weight1 neighbor2:weight2 ...
+ */
+public class NormalizingLongDoubleDoubleTextInputFormat
+    extends
+    TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable>
+    implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
+    DoubleWritable, Writable> {
+  /** Configuration. */
+  private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
+      DoubleWritable, Writable> conf;
+
+  @Override
+  public TextVertexReader createVertexReader(
+      InputSplit split, TaskAttemptContext context) throws IOException {
+    return new NormalizingLongDoubleDoubleDoubleVertexReader();
+  }
+
+  @Override
+  public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
+      DoubleWritable, DoubleWritable, Writable> configuration) {
+    conf = configuration;
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
+      DoubleWritable, Writable> getConf() {
+    return conf;
+  }
+
+  /**
+   * Vertex reader associated with
+   * {@link LongDoubleDoubleTextInputFormat}.
+   */
+  public class NormalizingLongDoubleDoubleDoubleVertexReader
+      extends NormalizingLongDoubleDoubleTextInputFormat.TextVertexReader {
+    /** Separator of the vertex and neighbors */
+    private final Pattern edgeSeparator = Pattern.compile("\\s+");
+    /** Separator of the edge id and edge weight */
+    private final Pattern weightSeparator = Pattern.compile(":");
+
+    @Override
+    public Vertex<LongWritable, DoubleWritable,
+        DoubleWritable, ?> getCurrentVertex()
+      throws IOException, InterruptedException {
+      Vertex<LongWritable, DoubleWritable,
+      DoubleWritable, ?> vertex = conf.createVertex();
+
+      String[] tokens = edgeSeparator.split(getRecordReader()
+          .getCurrentValue().toString());
+      List<Edge<LongWritable, DoubleWritable>> edges = Lists
+          .newArrayListWithCapacity(tokens.length - 1);
+      parse(tokens, edges);
+      normalize(edges);
+
+      LongWritable vertexId = new LongWritable(Long.parseLong(tokens[0]));
+      vertex.initialize(vertexId, new DoubleWritable(), edges);
+
+      return vertex;
+    }
+
+    /**
+     * Parse a set of tokens into a map ID -> weight.
+     * @param tokens The tokens to be parsed.
+     * @param edges The map that will contain the result of the parsing.
+     */
+    void parse(String[] tokens,
+               Collection<Edge<LongWritable, DoubleWritable>> edges) {
+      for (int n = 1; n < tokens.length; n++) {
+        String[] parts = weightSeparator.split(tokens[n]);
+        edges.add(EdgeFactory.create(new LongWritable(Long.parseLong(parts[0])),
+            new DoubleWritable(Double.parseDouble(parts[1]))));
+      }
+    }
+
+    /**
+     * Normalize the edges with L1 normalization.
+     * @param edges The edges to be normalized.
+     */
+    void normalize(Collection<Edge<LongWritable, DoubleWritable>> edges) {
+      if (edges == null || edges.size() == 0) {
+        throw new IllegalArgumentException(
+            "Cannot normalize an empy set of edges");
+      }
+      float normalizer = 0.0f;
+      for (Edge<LongWritable, DoubleWritable> edge : edges) {
+        normalizer += edge.getValue().get();
+      }
+      for (Edge<LongWritable, DoubleWritable> edge : edges) {
+        edge.getValue().set(edge.getValue().get() / normalizer);
+      }
+    }
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+      return getRecordReader().nextKeyValue();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
index 224b8f2..73dc9a9 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
@@ -170,8 +170,7 @@ public class SimplePageRankVertex extends Vertex<LongWritable,
    * Simple VertexReader that supports {@link SimplePageRankVertex}
    */
   public static class SimplePageRankVertexReader extends
-      GeneratedVertexReader<LongWritable, DoubleWritable, FloatWritable,
-      DoubleWritable> {
+      GeneratedVertexReader<LongWritable, DoubleWritable, FloatWritable> {
     /** Class logger */
     private static final Logger LOG =
         Logger.getLogger(SimplePageRankVertexReader.class);
@@ -212,11 +211,10 @@ public class SimplePageRankVertex extends Vertex<LongWritable,
    * Simple VertexInputFormat that supports {@link SimplePageRankVertex}
    */
   public static class SimplePageRankVertexInputFormat extends
-    GeneratedVertexInputFormat<LongWritable,
-            DoubleWritable, FloatWritable, DoubleWritable> {
+    GeneratedVertexInputFormat<LongWritable, DoubleWritable, FloatWritable> {
     @Override
     public VertexReader<LongWritable, DoubleWritable,
-    FloatWritable, DoubleWritable> createVertexReader(InputSplit split,
+    FloatWritable> createVertexReader(InputSplit split,
       TaskAttemptContext context)
       throws IOException {
       return new SimplePageRankVertexReader();

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
index 8313a2b..df0359a 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
@@ -67,8 +67,7 @@ public class SimpleSuperstepVertex extends Vertex<LongWritable, IntWritable,
    * Simple VertexReader that supports {@link SimpleSuperstepVertex}
    */
   public static class SimpleSuperstepVertexReader extends
-      GeneratedVertexReader<LongWritable, IntWritable,
-        FloatWritable, IntWritable> {
+      GeneratedVertexReader<LongWritable, IntWritable, FloatWritable> {
     /** Class logger */
     private static final Logger LOG =
         Logger.getLogger(SimpleSuperstepVertexReader.class);
@@ -114,10 +113,9 @@ public class SimpleSuperstepVertex extends Vertex<LongWritable, IntWritable,
    * Simple VertexInputFormat that supports {@link SimpleSuperstepVertex}
    */
   public static class SimpleSuperstepVertexInputFormat extends
-    GeneratedVertexInputFormat<LongWritable,
-        IntWritable, FloatWritable, IntWritable> {
+    GeneratedVertexInputFormat<LongWritable, IntWritable, FloatWritable> {
     @Override
-    public VertexReader<LongWritable, IntWritable, FloatWritable, IntWritable>
+    public VertexReader<LongWritable, IntWritable, FloatWritable>
     createVertexReader(InputSplit split, TaskAttemptContext context)
       throws IOException {
       return new SimpleSuperstepVertexReader();

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 666a50b..ab88b88 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
@@ -129,7 +129,7 @@ public class
         configuration.createVertex();
     vertex.initialize(new LongWritable(1), new IntWritable(1));
     System.out.println("testInstantiateVertex: Got vertex " + vertex);
-    VertexInputFormat<LongWritable, IntWritable, FloatWritable, IntWritable>
+    VertexInputFormat<LongWritable, IntWritable, FloatWritable>
     inputFormat = configuration.createVertexInputFormat();
 /*if[HADOOP_NON_JOBCONTEXT_IS_INTERFACE]
       List<InputSplit> splitArray =

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
index 5cc25be..55ca60c 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTest.java
@@ -26,7 +26,7 @@ import com.google.common.collect.SetMultimap;
 import org.apache.giraph.combiner.MinimumIntCombiner;
 import org.apache.giraph.conf.GiraphClasses;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
-import org.apache.giraph.io.formats.IntIntNullIntTextInputFormat;
+import org.apache.giraph.io.formats.IntIntNullTextInputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.junit.Test;
@@ -70,7 +70,7 @@ public class ConnectedComponentsVertexTest {
         classes.setVertexClass(ConnectedComponentsVertex.class);
         classes.setVertexEdgesClass(ByteArrayEdges.class);
         classes.setCombinerClass(MinimumIntCombiner.class);
-        classes.setVertexInputFormatClass(IntIntNullIntTextInputFormat.class);
+        classes.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
         classes.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
         Map<String, String> emptyParams = ImmutableMap.of();
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
index 27cd267..e42f172 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsVertexTestInMemory.java
@@ -18,25 +18,17 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.base.Splitter;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.SetMultimap;
 import org.apache.giraph.combiner.MinimumIntCombiner;
 import org.apache.giraph.conf.GiraphClasses;
 import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
-import org.apache.giraph.io.formats.IntIntNullIntTextInputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.giraph.utils.TestGraph;
 import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.edge.VertexEdges;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.log4j.Logger;
 import org.junit.Test;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
index 9672d20..042dbe4 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/PageRankVertexTest.java
@@ -63,7 +63,7 @@ public class PageRankVertexTest {
     classes.setVertexClass(PageRankVertex.class);
     classes.setVertexEdgesClass(ByteArrayEdges.class);
     classes.setVertexInputFormatClass(
-        LongDoubleNullDoubleTextInputFormat.class);
+        LongDoubleNullTextInputFormat.class);
     classes.setVertexOutputFormatClass(
         VertexWithDoubleValueNullEdgeTextOutputFormat.class);
     classes.setWorkerContextClass(RandomWalkWorkerContext.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
index 1ae9c52..f77ea3e 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/RandomWalkWithRestartVertexTest.java
@@ -57,7 +57,7 @@ public class RandomWalkWithRestartVertexTest {
     classes.setVertexClass(RandomWalkWithRestartVertex.class);
     classes.setVertexEdgesClass(ByteArrayEdges.class);
     classes.setVertexInputFormatClass(
-        LongDoubleDoubleDoubleTextInputFormat.class);
+        LongDoubleDoubleTextInputFormat.class);
     classes.setVertexOutputFormatClass(
         VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
     classes.setWorkerContextClass(RandomWalkWorkerContext.class);
@@ -98,7 +98,7 @@ public class RandomWalkWithRestartVertexTest {
     classes.setVertexClass(RandomWalkWithRestartVertex.class);
     classes.setVertexEdgesClass(ByteArrayEdges.class);
     classes.setVertexInputFormatClass(
-        NormalizingLongDoubleDoubleDoubleTextInputFormat.class);
+        NormalizingLongDoubleDoubleTextInputFormat.class);
     classes.setVertexOutputFormatClass(
         VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
     classes.setWorkerContextClass(RandomWalkWorkerContext.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 c8ca8a1..48153ae 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
@@ -22,7 +22,7 @@ import org.apache.giraph.combiner.MinimumIntCombiner;
 import org.apache.giraph.conf.GiraphClasses;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
-import org.apache.giraph.io.formats.IntIntNullIntTextInputFormat;
+import org.apache.giraph.io.formats.IntIntNullTextInputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.junit.Test;
@@ -78,7 +78,7 @@ public class TryMultiIpcBindingPortsTest {
         classes.setVertexClass(ConnectedComponentsVertex.class);
         classes.setVertexEdgesClass(ByteArrayEdges.class);
         classes.setCombinerClass(MinimumIntCombiner.class);
-        classes.setVertexInputFormatClass(IntIntNullIntTextInputFormat.class);
+        classes.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
         classes.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
 
         Iterable<String> results = InternalVertexRunner.run(classes, params,

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-hbase/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hbase/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java b/giraph-hbase/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java
index bbcbc1b..590ecb0 100644
--- a/giraph-hbase/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java
+++ b/giraph-hbase/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java
@@ -51,15 +51,13 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public abstract class HBaseVertexInputFormat<
     I extends WritableComparable,
     V extends Writable,
-    E extends Writable,
-    M extends Writable>
-    extends VertexInputFormat<I, V, E, M>  {
+    E extends Writable>
+    extends VertexInputFormat<I, V, E>  {
 
 
    /**
@@ -85,15 +83,15 @@ public abstract class HBaseVertexInputFormat<
    * @param <I> Vertex index value
    * @param <V> Vertex value
    * @param <E> Edge value
-   * @param <M> Message data
    */
   public abstract static class HBaseVertexReader<
           I extends WritableComparable,
           V extends Writable,
-          E extends Writable, M extends Writable>
-          implements VertexReader<I, V, E, M> {
+          E extends Writable>
+          implements VertexReader<I, V, E> {
     /** Giraph configuration */
-    private ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+    private ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    configuration;
     /** Reader instance */
     private final RecordReader<ImmutableBytesWritable, Result> reader;
     /** Context passed to initialize */
@@ -112,7 +110,8 @@ public abstract class HBaseVertexInputFormat<
       this.reader = BASE_FORMAT.createRecordReader(split, context);
     }
 
-    public ImmutableClassesGiraphConfiguration<I, V, E, M> getConfiguration() {
+    public ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    getConfiguration() {
       return configuration;
     }
 
@@ -130,8 +129,9 @@ public abstract class HBaseVertexInputFormat<
       InterruptedException {
       reader.initialize(inputSplit, context);
       this.context = context;
-      this.configuration = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
-          context.getConfiguration());
+      this.configuration =
+          new ImmutableClassesGiraphConfiguration<I, V, E, Writable>(
+              context.getConfiguration());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 8e209af..55e506b 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
@@ -38,13 +38,13 @@ import java.util.List;
   children qualifier to create an edge.
  */
 public class TableEdgeInputFormat extends
-        HBaseVertexInputFormat<Text, Text, Text, Text> {
+        HBaseVertexInputFormat<Text, Text, Text> {
 
     private static final Logger log =
             Logger.getLogger(TableEdgeInputFormat.class);
     private static final Text uselessEdgeValue = new Text();
 
-    public VertexReader<Text, Text, Text, Text>
+    public VertexReader<Text, Text, Text>
             createVertexReader(InputSplit split,
                                TaskAttemptContext context) throws IOException {
 
@@ -56,7 +56,7 @@ public class TableEdgeInputFormat extends
      Uses the RecordReader to return Hbase rows
      */
     public static class TableEdgeVertexReader
-            extends HBaseVertexReader<Text, Text, Text, Text> {
+            extends HBaseVertexReader<Text, Text, Text> {
 
         private final byte[] CF = Bytes.toBytes("cf");
         private final byte[] CHILDREN = Bytes.toBytes("children");
@@ -75,11 +75,11 @@ public class TableEdgeInputFormat extends
          For each row, create a vertex with the row ID as a text,
          and it's 'children' qualifier as a single edge.
          */
-        public Vertex<Text, Text, Text, Text>
+        public Vertex<Text, Text, Text, ?>
                     getCurrentVertex()
                 throws IOException, InterruptedException {
             Result row = getRecordReader().getCurrentValue();
-            Vertex<Text, Text, Text, Text> vertex =
+            Vertex<Text, Text, Text, ?> vertex =
                 getConfiguration().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/c5c1c4c6/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 fd5fb5c..9dea18e 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
@@ -55,16 +55,14 @@ import java.util.List;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 
 @SuppressWarnings("rawtypes")
 public abstract class HCatalogVertexInputFormat<
     I extends WritableComparable,
     V extends Writable,
-    E extends Writable,
-    M extends Writable>
-    extends VertexInputFormat<I, V, E, M> {
+    E extends Writable>
+    extends VertexInputFormat<I, V, E> {
   /**
    * HCatalog input format.
    */
@@ -87,16 +85,18 @@ public abstract class HCatalogVertexInputFormat<
    * nextVertex() should be overwritten to handle that logic as well.
    */
   protected abstract class HCatalogVertexReader implements
-      VertexReader<I, V, E, M> {
+      VertexReader<I, V, E> {
     /** Giraph configuration */
-    private ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+    private ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    configuration;
     /** Internal HCatRecordReader. */
     private RecordReader<WritableComparable,
         HCatRecord> hCatRecordReader;
     /** Context passed to initialize. */
     private TaskAttemptContext context;
 
-    public ImmutableClassesGiraphConfiguration<I, V, E, M> getConfiguration() {
+    public ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    getConfiguration() {
       return configuration;
     }
 
@@ -119,8 +119,9 @@ public abstract class HCatalogVertexInputFormat<
       throws IOException, InterruptedException {
       hCatRecordReader.initialize(inputSplit, ctxt);
       this.context = ctxt;
-      this.configuration = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
-          context.getConfiguration());
+      this.configuration =
+          new ImmutableClassesGiraphConfiguration<I, V, E, Writable>(
+              context.getConfiguration());
     }
 
     @Override
@@ -169,7 +170,7 @@ public abstract class HCatalogVertexInputFormat<
   protected abstract HCatalogVertexReader createVertexReader();
 
   @Override
-  public final VertexReader<I, V, E, M>
+  public final VertexReader<I, V, E>
   createVertexReader(final InputSplit split,
                      final TaskAttemptContext context)
     throws IOException {
@@ -237,10 +238,10 @@ public abstract class HCatalogVertexInputFormat<
     protected abstract Iterable<Edge<I, E>> getEdges(HCatRecord record);
 
     @Override
-    public final Vertex<I, V, E, M> getCurrentVertex()
+    public final Vertex<I, V, E, ?> getCurrentVertex()
       throws IOException, InterruptedException {
       HCatRecord record = getRecordReader().getCurrentValue();
-      Vertex<I, V, E, M> vertex = getConfiguration().createVertex();
+      Vertex<I, V, E, ?> vertex = getConfiguration().createVertex();
       vertex.initialize(getVertexId(record), getVertexValue(record),
           getEdges(record));
       ++recordCount;
@@ -299,7 +300,7 @@ public abstract class HCatalogVertexInputFormat<
     /**
      * vertex.
      */
-    private Vertex<I, V, E, M> vertex = null;
+    private Vertex<I, V, E, ?> vertex = null;
     /**
      * Timed logger to print every 30 seconds
      */
@@ -340,7 +341,7 @@ public abstract class HCatalogVertexInputFormat<
     protected abstract E getEdgeValue(HCatRecord record);
 
     @Override
-    public final Vertex<I, V, E, M>
+    public final Vertex<I, V, E, ?>
     getCurrentVertex() throws IOException, InterruptedException {
       return vertex;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexValueInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexValueInputFormat.java b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexValueInputFormat.java
index f365578..928e40c 100644
--- a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexValueInputFormat.java
+++ b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexValueInputFormat.java
@@ -37,15 +37,10 @@ import java.util.List;
  *
  * @param <I> Vertex id
  * @param <V> Vertex value
- * @param <E> Edge value
- * @param <M> Message data
  */
 public abstract class HCatalogVertexValueInputFormat<I extends
-    WritableComparable,
-    V extends Writable,
-    E extends Writable,
-    M extends Writable>
-    extends VertexValueInputFormat<I, V, E, M> {
+    WritableComparable, V extends Writable>
+    extends VertexValueInputFormat<I, V> {
   /**
    * HCatalog input format.
    */
@@ -61,7 +56,7 @@ public abstract class HCatalogVertexValueInputFormat<I extends
    * {@link VertexValueReader} for {@link HCatalogVertexValueInputFormat}.
    */
   protected abstract class HCatalogVertexValueReader
-      extends VertexValueReader<I, V, E, M> {
+      extends VertexValueReader<I, V> {
     /** Internal {@link RecordReader}. */
     private RecordReader<WritableComparable, HCatRecord> hCatRecordReader;
     /** Context passed to initialize. */
@@ -121,7 +116,7 @@ public abstract class HCatalogVertexValueInputFormat<I extends
   protected abstract HCatalogVertexValueReader createVertexValueReader();
 
   @Override
-  public final VertexValueReader<I, V, E, M>
+  public final VertexValueReader<I, V>
   createVertexValueReader(InputSplit split, TaskAttemptContext context)
     throws IOException {
     try {

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 c16e808..73fc553 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
@@ -17,6 +17,7 @@
  */
 package org.apache.giraph.hive.input.edge;
 
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.hive.common.DefaultConfigurableAndTableSchemaAware;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -25,15 +26,19 @@ import org.apache.hadoop.io.WritableComparable;
  * Base class for HiveToEdge implementations
  *
  * @param <I> Vertex ID
- * @param <V> Vertex Value
  * @param <E> Edge Value
- * @param <M> Message Value
  */
 public abstract class AbstractHiveToEdge<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends DefaultConfigurableAndTableSchemaAware<I, V, E, M>
+    E extends Writable>
+    extends DefaultConfigurableAndTableSchemaAware<I, Writable, E, Writable>
     implements HiveToEdge<I, E> {
   @Override
+  public ImmutableClassesGiraphConfiguration<I, Writable, E, Writable>
+  getConf() {
+    return super.getConf();
+  }
+
+  @Override
   public final void remove() {
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 e1a69cf..9093603 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
@@ -19,6 +19,7 @@
 package org.apache.giraph.hive.input.edge;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.hive.common.DefaultConfigurableAndTableSchemaAware;
 import org.apache.giraph.hive.input.RecordReaderWrapper;
 import org.apache.giraph.io.iterables.EdgeWithSource;
 import org.apache.giraph.io.iterables.GiraphReader;
@@ -30,8 +31,6 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
 import com.facebook.giraph.hive.record.HiveReadableRecord;
-import com.facebook.giraph.hive.schema.HiveTableSchema;
-import com.facebook.giraph.hive.schema.HiveTableSchemaAware;
 import com.facebook.giraph.hive.schema.HiveTableSchemas;
 
 import java.io.IOException;
@@ -45,17 +44,10 @@ import static org.apache.giraph.hive.common.GiraphHiveConstants.HIVE_TO_EDGE_CLA
  * @param <E> Edge Value
  */
 public class HiveEdgeReader<I extends WritableComparable, E extends Writable>
-    implements GiraphReader<EdgeWithSource<I, E>>, HiveTableSchemaAware {
-  /** Configuration key for edge creator class */
-  public static final String HIVE_TO_EDGE_KEY = "giraph.hive.to.edge.class";
-
+    extends DefaultConfigurableAndTableSchemaAware<I, Writable, E, Writable>
+    implements GiraphReader<EdgeWithSource<I, E>> {
   /** Underlying Hive RecordReader used */
   private RecordReader<WritableComparable, HiveReadableRecord> hiveRecordReader;
-  /** Schema for table in Hive */
-  private HiveTableSchema tableSchema;
-
-  /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, ?, E, ?> conf;
 
   /** User class to create edges from a HiveRecord */
   private HiveToEdge<I, E> hiveToEdge;
@@ -81,29 +73,11 @@ public class HiveEdgeReader<I extends WritableComparable, E extends Writable>
   }
 
   @Override
-  public HiveTableSchema getTableSchema() {
-    return tableSchema;
-  }
-
-  @Override
-  public void setTableSchema(HiveTableSchema tableSchema) {
-    this.tableSchema = tableSchema;
-  }
-
-  /**
-   * Get our Configuration.
-   *
-   * @return ImmutableClassesGiraphConfiguration
-   */
-  public ImmutableClassesGiraphConfiguration<I, ?, E, ?> getConf() {
-    return conf;
-  }
-
-  @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext context)
     throws IOException, InterruptedException {
     hiveRecordReader.initialize(inputSplit, context);
-    conf = new ImmutableClassesGiraphConfiguration(context.getConfiguration());
+    setConf(new ImmutableClassesGiraphConfiguration<I, Writable, E, Writable>(
+        context.getConfiguration()));
     instantiateHiveToEdgeFromConf();
     hiveToEdge.initializeRecords(
         new RecordReaderWrapper<HiveReadableRecord>(hiveRecordReader));
@@ -115,12 +89,12 @@ public class HiveEdgeReader<I extends WritableComparable, E extends Writable>
    * @throws IOException if anything goes wrong reading from Configuration
    */
   private void instantiateHiveToEdgeFromConf() throws IOException {
-    Class<? extends HiveToEdge> klass = HIVE_TO_EDGE_CLASS.get(conf);
+    Class<? extends HiveToEdge> klass = HIVE_TO_EDGE_CLASS.get(getConf());
     if (klass == null) {
       throw new IOException(HIVE_TO_EDGE_CLASS.getKey() + " not set in conf");
     }
-    hiveToEdge = ReflectionUtils.newInstance(klass, conf);
-    HiveTableSchemas.configure(hiveToEdge, tableSchema);
+    hiveToEdge = ReflectionUtils.newInstance(klass, getConf());
+    HiveTableSchemas.configure(hiveToEdge, getTableSchema());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/SimpleHiveToEdge.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/SimpleHiveToEdge.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/SimpleHiveToEdge.java
index 94a811e..d27803e 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/SimpleHiveToEdge.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/SimpleHiveToEdge.java
@@ -31,13 +31,10 @@ import java.util.Iterator;
  * row of the input.
  *
  * @param <I> Vertex id
- * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public abstract class SimpleHiveToEdge<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends AbstractHiveToEdge<I, V, E, M> {
+    E extends Writable> extends AbstractHiveToEdge<I, E> {
   /** Iterator over input records */
   private Iterator<HiveReadableRecord> records;
   /** Reusable {@link EdgeWithSource} object */

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 a138846..b08a09f 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
@@ -18,6 +18,7 @@
 
 package org.apache.giraph.hive.input.vertex;
 
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.hive.common.DefaultConfigurableAndTableSchemaAware;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -28,12 +29,16 @@ import org.apache.hadoop.io.WritableComparable;
  * @param <I> Vertex ID
  * @param <V> Vertex Value
  * @param <E> Edge Value
- * @param <M> Message Value
  */
 public abstract class AbstractHiveToVertex<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends DefaultConfigurableAndTableSchemaAware<I, V, E, M>
-    implements HiveToVertex<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends DefaultConfigurableAndTableSchemaAware<I, V, E, Writable>
+    implements HiveToVertex<I, V, E> {
+  @Override
+  public ImmutableClassesGiraphConfiguration<I, V, E, Writable> getConf() {
+    return super.getConf();
+  }
+
   @Override
   public final void remove() {
     throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 f57dc4e..595da85 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
@@ -36,11 +36,10 @@ import java.util.Iterator;
  * @param <I> Vertex ID
  * @param <V> Vertex Value
  * @param <E> Edge Value
- * @param <M> Message Value
  */
 public interface HiveToVertex<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> extends
-    Iterator<Vertex<I, V, E, M>> {
+    V extends Writable, E extends Writable> extends
+    Iterator<Vertex<I, V, E, ?>> {
   /**
    * Set the records which contain vertex input data
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 3b25444..1f87257 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
@@ -42,11 +42,10 @@ import java.util.List;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 public class HiveVertexInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends VertexInputFormat<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends VertexInputFormat<I, V, E> {
   /** Underlying Hive InputFormat used */
   private final HiveApiInputFormat hiveInputFormat;
 
@@ -65,11 +64,11 @@ public class HiveVertexInputFormat<I extends WritableComparable,
   }
 
   @Override
-  public VertexReader<I, V, E, M> createVertexReader(InputSplit split,
+  public VertexReader<I, V, E> createVertexReader(InputSplit split,
       TaskAttemptContext context) throws IOException {
     Configuration conf = context.getConfiguration();
 
-    HiveVertexReader<I, V, E, M> reader = new HiveVertexReader<I, V, E, M>();
+    HiveVertexReader<I, V, E> reader = new HiveVertexReader<I, V, E>();
     reader.setTableSchema(hiveInputFormat.getTableSchema(conf));
 
     RecordReader<WritableComparable, HiveReadableRecord> baseReader;
@@ -80,6 +79,6 @@ public class HiveVertexInputFormat<I extends WritableComparable,
     }
 
     reader.setHiveRecordReader(baseReader);
-    return new VertexReaderWrapper<I, V, E, M>(reader);
+    return new VertexReaderWrapper<I, V, E>(reader);
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 9c172be..318e83f 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
@@ -20,6 +20,7 @@ package org.apache.giraph.hive.input.vertex;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.hive.common.DefaultConfigurableAndTableSchemaAware;
 import org.apache.giraph.hive.input.RecordReaderWrapper;
 import org.apache.giraph.io.iterables.GiraphReader;
 import org.apache.giraph.utils.ReflectionUtils;
@@ -30,8 +31,6 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
 import com.facebook.giraph.hive.record.HiveReadableRecord;
-import com.facebook.giraph.hive.schema.HiveTableSchema;
-import com.facebook.giraph.hive.schema.HiveTableSchemaAware;
 import com.facebook.giraph.hive.schema.HiveTableSchemas;
 
 import java.io.IOException;
@@ -44,24 +43,19 @@ import static org.apache.giraph.hive.common.GiraphHiveConstants.HIVE_TO_VERTEX_C
  * @param <I> Vertex ID
  * @param <V> Vertex Value
  * @param <E> Edge Value
- * @param <M> Message Value
  */
 public class HiveVertexReader<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements GiraphReader<Vertex<I, V, E, M>>, HiveTableSchemaAware {
+    V extends Writable, E extends Writable>
+    extends DefaultConfigurableAndTableSchemaAware<I, V, E, Writable>
+    implements GiraphReader<Vertex<I, V, E, ?>> {
   /** Underlying Hive RecordReader used */
   private RecordReader<WritableComparable, HiveReadableRecord> hiveRecordReader;
-  /** Schema for table in Hive */
-  private HiveTableSchema tableSchema;
-
-  /** Configuration */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
 
   /**
    * {@link HiveToVertex} chosen by user,
    * or {@link SimpleHiveToVertex} if none specified
    */
-  private HiveToVertex<I, V, E, M> hiveToVertex;
+  private HiveToVertex<I, V, E> hiveToVertex;
 
   /**
    * Get underlying Hive record reader used.
@@ -84,33 +78,14 @@ public class HiveVertexReader<I extends WritableComparable,
   }
 
   @Override
-  public HiveTableSchema getTableSchema() {
-    return tableSchema;
-  }
-
-  @Override
-  public void setTableSchema(HiveTableSchema tableSchema) {
-    this.tableSchema = tableSchema;
-  }
-
-  /**
-   * Get our Configuration.
-   *
-   * @return ImmutableClassesGiraphConfiguration
-   */
-  public ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
-    return conf;
-  }
-
-  @Override
   public void initialize(InputSplit inputSplit,
       TaskAttemptContext context) throws IOException, InterruptedException {
     hiveRecordReader.initialize(inputSplit, context);
-    conf = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
-        context.getConfiguration());
-    Class<? extends HiveToVertex> klass = HIVE_TO_VERTEX_CLASS.get(conf);
-    hiveToVertex = ReflectionUtils.newInstance(klass, conf);
-    HiveTableSchemas.configure(hiveToVertex, tableSchema);
+    setConf(new ImmutableClassesGiraphConfiguration<I, V, E, Writable>(
+        context.getConfiguration()));
+    Class<? extends HiveToVertex> klass = HIVE_TO_VERTEX_CLASS.get(getConf());
+    hiveToVertex = ReflectionUtils.newInstance(klass, getConf());
+    HiveTableSchemas.configure(hiveToVertex, getTableSchema());
     hiveToVertex.initializeRecords(
         new RecordReaderWrapper<HiveReadableRecord>(hiveRecordReader));
   }
@@ -131,7 +106,7 @@ public class HiveVertexReader<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, M> next() {
+  public Vertex<I, V, E, ?> next() {
     return hiveToVertex.next();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 59c10be..21a47ee 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
@@ -35,16 +35,15 @@ import java.util.Iterator;
  * @param <I> Vertex ID
  * @param <V> Vertex Value
  * @param <E> Edge Value
- * @param <M> Message Value
  */
 public abstract class SimpleHiveToVertex<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends AbstractHiveToVertex<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends AbstractHiveToVertex<I, V, E> {
   /** Hive records which we are reading from */
   private Iterator<HiveReadableRecord> records;
 
   /** Reusable vertex object */
-  private Vertex<I, V, E, M> reusableVertex = null;
+  private Vertex<I, V, E, ?> reusableVertex = null;
 
   /**
    * Read the Vertex's ID from the HiveRecord given.
@@ -71,7 +70,8 @@ public abstract class SimpleHiveToVertex<I extends WritableComparable,
   public abstract Iterable<Edge<I, E>> getEdges(HiveReadableRecord record);
 
   @Override
-  public void setConf(ImmutableClassesGiraphConfiguration<I, V, E, M> conf) {
+  public void setConf(
+      ImmutableClassesGiraphConfiguration<I, V, E, Writable> 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, M> 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, M> vertex = reusableVertex;
+    Vertex<I, V, E, ?> vertex = reusableVertex;
     if (vertex == null) {
       vertex = getConf().createVertex();
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleNoEdgesHiveToVertex.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleNoEdgesHiveToVertex.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleNoEdgesHiveToVertex.java
index ca427a0..d8dfc37 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleNoEdgesHiveToVertex.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/SimpleNoEdgesHiveToVertex.java
@@ -31,14 +31,11 @@ import com.google.common.collect.ImmutableList;
  *
  * @param <I> Vertex ID
  * @param <V> Vertex Value
- * @param <E> Edge Value
- * @param <M> Message Value
  */
 public abstract class SimpleNoEdgesHiveToVertex<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> extends
-    SimpleHiveToVertex<I, V, E, M> {
+    V extends Writable> extends SimpleHiveToVertex<I, V, Writable> {
   @Override
-  public Iterable<Edge<I, E>> getEdges(HiveReadableRecord record) {
+  public final Iterable<Edge<I, Writable>> getEdges(HiveReadableRecord record) {
     return ImmutableList.of();
   }
 }


[2/2] git commit: updated refs/heads/trunk to c5c1c4c

Posted by ma...@apache.org.
GIRAPH-589: Remove unnecessary generics from input formats (majakabiljo)


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

Branch: refs/heads/trunk
Commit: c5c1c4c6131ee67265c4c322d53b689f9a52dc40
Parents: ab88282
Author: Maja Kabiljo <ma...@maja-mbp.thefacebook.com>
Authored: Wed Apr 3 11:17:52 2013 -0700
Committer: Maja Kabiljo <ma...@maja-mbp.thefacebook.com>
Committed: Wed Apr 3 12:13:01 2013 -0700

----------------------------------------------------------------------
 .../io/accumulo/AccumuloVertexInputFormat.java     |   22 ++--
 .../edgemarker/AccumuloEdgeInputFormat.java        |   10 +-
 .../java/org/apache/giraph/conf/GiraphClasses.java |    8 +-
 .../conf/ImmutableClassesGiraphConfiguration.java  |    7 +-
 .../apache/giraph/io/BasicVertexValueReader.java   |    6 +-
 .../org/apache/giraph/io/VertexInputFormat.java    |    5 +-
 .../java/org/apache/giraph/io/VertexReader.java    |    5 +-
 .../apache/giraph/io/VertexValueInputFormat.java   |    9 +-
 .../org/apache/giraph/io/VertexValueReader.java    |   20 +-
 .../AdjacencyListTextVertexInputFormat.java        |    5 +-
 .../io/formats/GeneratedVertexInputFormat.java     |    5 +-
 .../io/formats/IntIntNullIntTextInputFormat.java   |   92 ----------
 .../io/formats/IntIntNullTextInputFormat.java      |   91 ++++++++++
 .../io/formats/IntNullNullNullTextInputFormat.java |   74 --------
 .../io/formats/IntNullNullTextInputFormat.java     |   74 ++++++++
 .../io/formats/JsonBase64VertexInputFormat.java    |    5 +-
 ...JsonLongDoubleFloatDoubleVertexInputFormat.java |    3 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |    7 +-
 .../io/formats/PseudoRandomVertexInputFormat.java  |   19 +--
 .../io/formats/SequenceFileVertexInputFormat.java  |   18 +--
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |    2 +-
 .../giraph/io/formats/TextVertexInputFormat.java   |   27 ++--
 .../io/formats/TextVertexValueInputFormat.java     |    4 +-
 .../giraph/io/iterables/VertexReaderWrapper.java   |   14 +-
 .../giraph/job/GiraphConfigurationValidator.java   |    2 +-
 .../org/apache/giraph/master/BspServiceMaster.java |    2 +-
 .../giraph/utils/InMemoryVertexInputFormat.java    |   16 +-
 .../giraph/worker/VertexInputSplitsCallable.java   |    6 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |    6 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |   23 +--
 .../apache/giraph/master/TestMasterObserver.java   |    4 +-
 .../giraph/examples/GeneratedVertexReader.java     |    5 +-
 .../LongDoubleDoubleDoubleTextInputFormat.java     |  108 -----------
 .../examples/LongDoubleDoubleTextInputFormat.java  |  109 +++++++++++
 .../LongDoubleNullDoubleTextInputFormat.java       |  106 -----------
 .../examples/LongDoubleNullTextInputFormat.java    |  106 +++++++++++
 ...izingLongDoubleDoubleDoubleTextInputFormat.java |  139 ---------------
 ...NormalizingLongDoubleDoubleTextInputFormat.java |  139 +++++++++++++++
 .../giraph/examples/SimplePageRankVertex.java      |    8 +-
 .../giraph/examples/SimpleSuperstepVertex.java     |    8 +-
 .../test/java/org/apache/giraph/TestBspBasic.java  |    2 +-
 .../examples/ConnectedComponentsVertexTest.java    |    4 +-
 .../ConnectedComponentsVertexTestInMemory.java     |    8 -
 .../apache/giraph/examples/PageRankVertexTest.java |    2 +-
 .../examples/RandomWalkWithRestartVertexTest.java  |    4 +-
 .../examples/TryMultiIpcBindingPortsTest.java      |    4 +-
 .../giraph/io/hbase/HBaseVertexInputFormat.java    |   22 ++--
 .../io/hbase/edgemarker/TableEdgeInputFormat.java  |   10 +-
 .../io/hcatalog/HCatalogVertexInputFormat.java     |   29 ++--
 .../hcatalog/HCatalogVertexValueInputFormat.java   |   13 +-
 .../giraph/hive/input/edge/AbstractHiveToEdge.java |   13 +-
 .../giraph/hive/input/edge/HiveEdgeReader.java     |   42 +----
 .../giraph/hive/input/edge/SimpleHiveToEdge.java   |    5 +-
 .../hive/input/vertex/AbstractHiveToVertex.java    |   13 +-
 .../giraph/hive/input/vertex/HiveToVertex.java     |    5 +-
 .../hive/input/vertex/HiveVertexInputFormat.java   |   11 +-
 .../giraph/hive/input/vertex/HiveVertexReader.java |   47 ++----
 .../hive/input/vertex/SimpleHiveToVertex.java      |   14 +-
 .../input/vertex/SimpleNoEdgesHiveToVertex.java    |    7 +-
 59 files changed, 739 insertions(+), 835 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 9445fd4..28dc951 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
@@ -44,14 +44,12 @@ import java.util.List;
  * @param <I> vertex id type
  * @param <V>  vertex value type
  * @param <E>  edge type
- * @param <M>  message type
  */
 public abstract class AccumuloVertexInputFormat<
         I extends WritableComparable,
         V extends Writable,
-        E extends Writable,
-        M extends Writable>
-        extends VertexInputFormat<I, V, E, M> {
+        E extends Writable>
+        extends VertexInputFormat<I, V, E> {
   /**
    * delegate input format for all accumulo operations.
    */
@@ -65,15 +63,15 @@ public abstract class AccumuloVertexInputFormat<
   * @param <I>  vertex id type
   * @param <V>  vertex value type
   * @param <E>  edge type
-  * @param <M>  message type
   */
   public abstract static class AccumuloVertexReader<
       I extends WritableComparable,
-      V extends Writable, E extends Writable, M extends Writable>
-      implements VertexReader<I, V, E, M> {
+      V extends Writable, E extends Writable>
+      implements VertexReader<I, V, E> {
 
     /** Giraph configuration */
-    private ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+    private ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    configuration;
     /**
      * Used by subclasses to read key/value pairs.
      */
@@ -89,7 +87,8 @@ public abstract class AccumuloVertexInputFormat<
       this.reader = reader;
     }
 
-    public ImmutableClassesGiraphConfiguration<I, V, E, M> getConfiguration() {
+    public ImmutableClassesGiraphConfiguration<I, V, E, Writable>
+    getConfiguration() {
       return configuration;
     }
 
@@ -106,8 +105,9 @@ public abstract class AccumuloVertexInputFormat<
       throws IOException, InterruptedException {
       reader.initialize(inputSplit, context);
       this.context = context;
-      this.configuration = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
-          context.getConfiguration());
+      this.configuration =
+          new ImmutableClassesGiraphConfiguration<I, V, E, Writable>(
+              context.getConfiguration());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 4cbbc89..4e76b74 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
@@ -39,11 +39,11 @@ import java.util.regex.Pattern;
  Example subclass which reads in Key/Value pairs to construct vertex objects.
  */
 public class AccumuloEdgeInputFormat
-        extends AccumuloVertexInputFormat<Text, Text, Text, Text> {
+        extends AccumuloVertexInputFormat<Text, Text, Text> {
 
     private static final Text uselessEdgeValue = new Text();
     private Configuration conf;
-    public VertexReader<Text, Text, Text, Text>
+    public VertexReader<Text, Text, Text>
     createVertexReader(InputSplit split, TaskAttemptContext context)
             throws IOException {
         try {
@@ -60,7 +60,7 @@ public class AccumuloEdgeInputFormat
         Reader takes Key/Value pairs from the underlying input format.
      */
     public static class AccumuloEdgeVertexReader
-            extends AccumuloVertexReader<Text, Text, Text, Text> {
+            extends AccumuloVertexReader<Text, Text, Text> {
 
         public static final Pattern commaPattern = Pattern.compile("[,]");
 
@@ -76,11 +76,11 @@ public class AccumuloEdgeInputFormat
         /*
        Each Key/Value contains the information needed to construct the vertices.
          */
-        public Vertex<Text, 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, 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/c5c1c4c6/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 e03859a..64f8bb1 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
@@ -82,7 +82,7 @@ public class GiraphClasses<I extends WritableComparable,
   graphPartitionerFactoryClass;
 
   /** Vertex input format class - cached for fast access */
-  protected Class<? extends VertexInputFormat<I, V, E, M>>
+  protected Class<? extends VertexInputFormat<I, V, E>>
   vertexInputFormatClass;
   /** Vertex output format class - cached for fast access */
   protected Class<? extends VertexOutputFormat<I, V, E>>
@@ -168,7 +168,7 @@ public class GiraphClasses<I extends WritableComparable,
         (Class<? extends GraphPartitionerFactory<I, V, E, M>>)
             GRAPH_PARTITIONER_FACTORY_CLASS.get(conf);
 
-    vertexInputFormatClass = (Class<? extends VertexInputFormat<I, V, E, M>>)
+    vertexInputFormatClass = (Class<? extends VertexInputFormat<I, V, E>>)
         VERTEX_INPUT_FORMAT_CLASS.get(conf);
     vertexOutputFormatClass = (Class<? extends VertexOutputFormat<I, V, E>>)
         VERTEX_OUTPUT_FORMAT_CLASS.get(conf);
@@ -283,7 +283,7 @@ public class GiraphClasses<I extends WritableComparable,
    *
    * @return VertexInputFormat
    */
-  public Class<? extends VertexInputFormat<I, V, E, M>>
+  public Class<? extends VertexInputFormat<I, V, E>>
   getVertexInputFormatClass() {
     return vertexInputFormatClass;
   }
@@ -566,7 +566,7 @@ public class GiraphClasses<I extends WritableComparable,
    * @return this
    */
   public GiraphClasses setVertexInputFormatClass(
-      Class<? extends VertexInputFormat<I, V, E, M>> vertexInputFormatClass) {
+      Class<? extends VertexInputFormat<I, V, E>> vertexInputFormatClass) {
     this.vertexInputFormatClass = vertexInputFormatClass;
     return this;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 e290c57..f6619c1 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
@@ -156,7 +156,7 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return User's vertex input format class
    */
-  public Class<? extends VertexInputFormat<I, V, E, M>>
+  public Class<? extends VertexInputFormat<I, V, E>>
   getVertexInputFormatClass() {
     return classes.getVertexInputFormatClass();
   }
@@ -166,9 +166,8 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    *
    * @return Instantiated user vertex input format class
    */
-  public VertexInputFormat<I, V, E, M>
-  createVertexInputFormat() {
-    Class<? extends VertexInputFormat<I, V, E, M>> klass =
+  public VertexInputFormat<I, V, E> createVertexInputFormat() {
+    Class<? extends VertexInputFormat<I, V, E>> klass =
         classes.getVertexInputFormatClass();
     return ReflectionUtils.newInstance(klass, this);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/BasicVertexValueReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/BasicVertexValueReader.java b/giraph-core/src/main/java/org/apache/giraph/io/BasicVertexValueReader.java
index 1ccde39..1aed963 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/BasicVertexValueReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/BasicVertexValueReader.java
@@ -28,12 +28,10 @@ import java.io.IOException;
  *
  * @param <I> Vertex id
  * @param <V> Vertex data
- * @param <E> Edge data
- * @param <M> Message data
  */
 public abstract class BasicVertexValueReader<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    implements VertexReader<I, V, E, M> {
+    V extends Writable>
+    implements VertexReader<I, V, Writable> {
   /**
    * User-defined method to extract the vertex id.
    *

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/VertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexInputFormat.java
index ded8b92..5aface5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexInputFormat.java
@@ -36,11 +36,10 @@ import java.util.List;
  * @param <I> Vertex id
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public abstract class VertexInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
+    V extends Writable, E extends Writable>
     implements GiraphInputFormat {
   @Override
   public abstract List<InputSplit> getSplits(
@@ -58,7 +57,7 @@ public abstract class VertexInputFormat<I extends WritableComparable,
    * @throws IOException
    * @throws InterruptedException
    */
-  public abstract VertexReader<I, V, E, M> createVertexReader(
+  public abstract VertexReader<I, V, E> createVertexReader(
       InputSplit split,
       TaskAttemptContext context) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 3487cee..ef4ded6 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
@@ -33,11 +33,10 @@ import java.io.IOException;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public interface VertexReader<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> {
+    V extends Writable, E extends Writable> {
   /**
    * Use the input split and context to setup reading the vertices.
    * Guaranteed to be called prior to any other function.
@@ -66,7 +65,7 @@ public interface VertexReader<I extends WritableComparable,
    * @throws IOException
    * @throws InterruptedException
    */
-  Vertex<I, V, E, M> getCurrentVertex()
+  Vertex<I, V, E, ?> getCurrentVertex()
     throws IOException, InterruptedException;
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/VertexValueInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexValueInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexValueInputFormat.java
index 1ebda13..29abe73 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexValueInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexValueInputFormat.java
@@ -31,12 +31,9 @@ import java.io.IOException;
  *
  * @param <I> Vertex id
  * @param <V> Vertex value
- * @param <E> Edge value
- * @param <M> Message value
  */
 public abstract class VertexValueInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends VertexInputFormat<I, V, E, M> {
+    V extends Writable> extends VertexInputFormat<I, V, Writable> {
   /**
    * Create a {@link VertexValueReader} for a given split. The framework will
    * call {@link VertexValueReader#initialize(InputSplit,
@@ -47,11 +44,11 @@ public abstract class VertexValueInputFormat<I extends WritableComparable,
    * @return A new vertex value reader
    * @throws IOException
    */
-  public abstract VertexValueReader<I, V, E, M> createVertexValueReader(
+  public abstract VertexValueReader<I, V> createVertexValueReader(
       InputSplit split, TaskAttemptContext context) throws IOException;
 
   @Override
-  public final VertexReader<I, V, E, M> createVertexReader(
+  public final VertexReader<I, V, Writable> createVertexReader(
       InputSplit split, TaskAttemptContext context) throws IOException {
     return createVertexValueReader(split, context);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 3ccb0fd..25e6093 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
@@ -32,31 +32,31 @@ import java.io.IOException;
  *
  * @param <I> Vertex id
  * @param <V> Vertex value
- * @param <E> Edge value
- * @param <M> Message value
  */
 public abstract class VertexValueReader<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends BasicVertexValueReader<I, V, E, M> {
+    V extends Writable> extends BasicVertexValueReader<I, V> {
   /** Configuration. */
-  private ImmutableClassesGiraphConfiguration<I, V, E, M> configuration;
+  private ImmutableClassesGiraphConfiguration<I, V, Writable, Writable>
+  configuration;
 
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext context)
     throws IOException, InterruptedException {
-    configuration = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
-        context.getConfiguration());
+    configuration =
+        new ImmutableClassesGiraphConfiguration<I, V, Writable, Writable>(
+            context.getConfiguration());
   }
 
   @Override
-  public final Vertex<I, V, E, M> getCurrentVertex() throws IOException,
+  public final Vertex<I, V, Writable, ?> getCurrentVertex() throws IOException,
       InterruptedException {
-    Vertex<I, V, E, M> vertex = getConf().createVertex();
+    Vertex<I, V, Writable, ?> vertex = getConf().createVertex();
     vertex.initialize(getCurrentVertexId(), getCurrentVertexValue());
     return vertex;
   }
 
-  public ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
+  public ImmutableClassesGiraphConfiguration<I, V, Writable, Writable>
+  getConf() {
     return configuration;
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexInputFormat.java
index 8fe0db6..7af4a71 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/AdjacencyListTextVertexInputFormat.java
@@ -40,12 +40,11 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public abstract class AdjacencyListTextVertexInputFormat<I extends
-    WritableComparable, V extends Writable, E extends Writable, M extends
-    Writable> extends TextVertexInputFormat<I, V, E, M> {
+    WritableComparable, V extends Writable, E extends Writable> extends
+    TextVertexInputFormat<I, V, E> {
   /** Delimiter for split */
   public static final String LINE_TOKENIZE_VALUE = "adj.list.input.delimiter";
   /** Default delimiter for split */

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/GeneratedVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/GeneratedVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/GeneratedVertexInputFormat.java
index 1ae125f..44cfd5c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/GeneratedVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/GeneratedVertexInputFormat.java
@@ -36,12 +36,11 @@ import java.util.List;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public abstract class GeneratedVertexInputFormat<
-    I extends WritableComparable, V extends Writable, E extends Writable,
-    M extends Writable> extends VertexInputFormat<I, V, E, M> {
+    I extends WritableComparable, V extends Writable, E extends Writable>
+    extends VertexInputFormat<I, V, E> {
   @Override
   public List<InputSplit> getSplits(JobContext context, int minSplitCountHint)
     throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullIntTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullIntTextInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullIntTextInputFormat.java
deleted file mode 100644
index 28539f5..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullIntTextInputFormat.java
+++ /dev/null
@@ -1,92 +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.io.formats;
-
-import com.google.common.collect.Lists;
-import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.EdgeFactory;
-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 java.io.IOException;
-import java.util.List;
-import java.util.regex.Pattern;
-
-/**
- * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
- * unweighted graphs with int ids.
- *
- * Each line consists of: vertex neighbor1 neighbor2 ...
- */
-public class IntIntNullIntTextInputFormat extends
-    TextVertexInputFormat<IntWritable, IntWritable, NullWritable,
-    IntWritable> {
-  /** Separator of the vertex and neighbors */
-  private static final Pattern SEPARATOR = Pattern.compile("[\t ]");
-
-  @Override
-  public TextVertexReader createVertexReader(InputSplit split,
-      TaskAttemptContext context)
-    throws IOException {
-    return new IntIntNullIntVertexReader();
-  }
-
-  /**
-   * Vertex reader associated with {@link IntIntNullIntTextInputFormat}.
-   */
-  public class IntIntNullIntVertexReader extends
-    TextVertexReaderFromEachLineProcessed<String[]> {
-    /**
-     * Cached vertex id for the current line
-     */
-    private IntWritable id;
-
-    @Override
-    protected String[] preprocessLine(Text line) throws IOException {
-      String[] tokens = SEPARATOR.split(line.toString());
-      id = new IntWritable(Integer.parseInt(tokens[0]));
-      return tokens;
-    }
-
-    @Override
-    protected IntWritable getId(String[] tokens) throws IOException {
-      return id;
-    }
-
-    @Override
-    protected IntWritable getValue(String[] tokens) throws IOException {
-      return id;
-    }
-
-    @Override
-    protected Iterable<Edge<IntWritable, NullWritable>> getEdges(
-        String[] tokens) throws IOException {
-      List<Edge<IntWritable, NullWritable>> edges =
-          Lists.newArrayListWithCapacity(tokens.length - 1);
-      for (int n = 1; n < tokens.length; n++) {
-        edges.add(EdgeFactory.create(
-            new IntWritable(Integer.parseInt(tokens[n]))));
-      }
-      return edges;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullTextInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullTextInputFormat.java
new file mode 100644
index 0000000..91725f8
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntIntNullTextInputFormat.java
@@ -0,0 +1,91 @@
+/*
+ * 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.io.formats;
+
+import com.google.common.collect.Lists;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.EdgeFactory;
+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 java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+/**
+ * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
+ * unweighted graphs with int ids.
+ *
+ * Each line consists of: vertex neighbor1 neighbor2 ...
+ */
+public class IntIntNullTextInputFormat extends
+    TextVertexInputFormat<IntWritable, IntWritable, NullWritable> {
+  /** Separator of the vertex and neighbors */
+  private static final Pattern SEPARATOR = Pattern.compile("[\t ]");
+
+  @Override
+  public TextVertexReader createVertexReader(InputSplit split,
+      TaskAttemptContext context)
+    throws IOException {
+    return new IntIntNullIntVertexReader();
+  }
+
+  /**
+   * Vertex reader associated with {@link IntIntNullTextInputFormat}.
+   */
+  public class IntIntNullIntVertexReader extends
+    TextVertexReaderFromEachLineProcessed<String[]> {
+    /**
+     * Cached vertex id for the current line
+     */
+    private IntWritable id;
+
+    @Override
+    protected String[] preprocessLine(Text line) throws IOException {
+      String[] tokens = SEPARATOR.split(line.toString());
+      id = new IntWritable(Integer.parseInt(tokens[0]));
+      return tokens;
+    }
+
+    @Override
+    protected IntWritable getId(String[] tokens) throws IOException {
+      return id;
+    }
+
+    @Override
+    protected IntWritable getValue(String[] tokens) throws IOException {
+      return id;
+    }
+
+    @Override
+    protected Iterable<Edge<IntWritable, NullWritable>> getEdges(
+        String[] tokens) throws IOException {
+      List<Edge<IntWritable, NullWritable>> edges =
+          Lists.newArrayListWithCapacity(tokens.length - 1);
+      for (int n = 1; n < tokens.length; n++) {
+        edges.add(EdgeFactory.create(
+            new IntWritable(Integer.parseInt(tokens[n]))));
+      }
+      return edges;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullNullTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullNullTextInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullNullTextInputFormat.java
deleted file mode 100644
index 4950d21..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullNullTextInputFormat.java
+++ /dev/null
@@ -1,74 +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.io.formats;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.giraph.edge.Edge;
-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 java.io.IOException;
-
-/**
- * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
- * unweighted graphs without edges or values, just vertices with ids.
- *
- * Each line is just simply the vertex id.
- */
-public class IntNullNullNullTextInputFormat extends TextVertexInputFormat<
-    IntWritable, NullWritable, NullWritable, NullWritable> {
-  @Override
-  public TextVertexReader createVertexReader(
-      InputSplit split, TaskAttemptContext context) throws IOException {
-    return new IntNullNullNullVertexReader();
-  }
-
-  /**
-   * Reader for this InputFormat.
-   */
-  public class IntNullNullNullVertexReader extends
-      TextVertexReaderFromEachLineProcessed<String> {
-    /** Cached vertex id */
-    private IntWritable id;
-
-    @Override
-    protected String preprocessLine(Text line) throws IOException {
-      id = new IntWritable(Integer.parseInt(line.toString()));
-      return line.toString();
-    }
-
-    @Override
-    protected IntWritable getId(String line) throws IOException {
-      return id;
-    }
-
-    @Override
-    protected NullWritable getValue(String line) throws IOException {
-      return NullWritable.get();
-    }
-
-    @Override
-    protected Iterable<Edge<IntWritable, NullWritable>> getEdges(String line)
-      throws IOException {
-      return ImmutableList.of();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullTextInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullTextInputFormat.java
new file mode 100644
index 0000000..b56c586
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullNullTextInputFormat.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.io.formats;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.giraph.edge.Edge;
+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 java.io.IOException;
+
+/**
+ * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
+ * unweighted graphs without edges or values, just vertices with ids.
+ *
+ * Each line is just simply the vertex id.
+ */
+public class IntNullNullTextInputFormat extends TextVertexInputFormat<
+    IntWritable, NullWritable, NullWritable> {
+  @Override
+  public TextVertexReader createVertexReader(
+      InputSplit split, TaskAttemptContext context) throws IOException {
+    return new IntNullNullNullVertexReader();
+  }
+
+  /**
+   * Reader for this InputFormat.
+   */
+  public class IntNullNullNullVertexReader extends
+      TextVertexReaderFromEachLineProcessed<String> {
+    /** Cached vertex id */
+    private IntWritable id;
+
+    @Override
+    protected String preprocessLine(Text line) throws IOException {
+      id = new IntWritable(Integer.parseInt(line.toString()));
+      return line.toString();
+    }
+
+    @Override
+    protected IntWritable getId(String line) throws IOException {
+      return id;
+    }
+
+    @Override
+    protected NullWritable getValue(String line) throws IOException {
+      return NullWritable.get();
+    }
+
+    @Override
+    protected Iterable<Edge<IntWritable, NullWritable>> getEdges(String line)
+      throws IOException {
+      return ImmutableList.of();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexInputFormat.java
index 6eaf7dc..bceb912 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonBase64VertexInputFormat.java
@@ -46,12 +46,11 @@ import java.util.List;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public class JsonBase64VertexInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends TextVertexInputFormat<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends TextVertexInputFormat<I, V, E> {
 
   @Override
   public TextVertexReader createVertexReader(InputSplit split,

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 2ac2dad..7dfd607 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
@@ -40,8 +40,7 @@ import java.util.List;
   *  specified in JSON format.
   */
 public class JsonLongDoubleFloatDoubleVertexInputFormat extends
-  TextVertexInputFormat<LongWritable, DoubleWritable,
-  FloatWritable, DoubleWritable> {
+  TextVertexInputFormat<LongWritable, DoubleWritable, FloatWritable> {
 
   @Override
   public TextVertexReader createVertexReader(InputSplit split,

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-core/src/main/java/org/apache/giraph/io/formats/LongDoubleDoubleAdjacencyListVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/LongDoubleDoubleAdjacencyListVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/LongDoubleDoubleAdjacencyListVertexInputFormat.java
index 09fb991..717975d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/LongDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/LongDoubleDoubleAdjacencyListVertexInputFormat.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.LongWritable;
-import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
@@ -32,12 +31,10 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * 22 0.1 45 0.3 99 0.44
  * to repesent a vertex with id 22, value of 0.1 and edges to nodes 45 and 99,
  * with values of 0.3 and 0.44, respectively.
- *
- * @param <M> Message data
  */
-public class LongDoubleDoubleAdjacencyListVertexInputFormat<M extends Writable>
+public class LongDoubleDoubleAdjacencyListVertexInputFormat
     extends AdjacencyListTextVertexInputFormat<LongWritable, DoubleWritable,
-    DoubleWritable, M> {
+    DoubleWritable> {
 
   @Override
   public AdjacencyListTextVertexReader createVertexReader(InputSplit split,

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 40a20e1..c261f72 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
@@ -28,7 +28,6 @@ import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.edge.VertexEdges;
 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.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
@@ -45,11 +44,9 @@ import java.util.Set;
  * to create an input data source that a variable number of aggregate vertices
  * and edges per vertex that is repeatable for the exact same parameter
  * (pseudo-random).
- *
- * @param <M> Message data
  */
-public class PseudoRandomVertexInputFormat<M extends Writable> extends
-    VertexInputFormat<LongWritable, DoubleWritable, DoubleWritable, M> {
+public class PseudoRandomVertexInputFormat extends
+    VertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
   @Override
   public final List<InputSplit> getSplits(final JobContext context,
       final int minSplitCountHint) throws IOException, InterruptedException {
@@ -63,18 +60,18 @@ public class PseudoRandomVertexInputFormat<M extends Writable> extends
   }
 
   @Override
-  public VertexReader<LongWritable, DoubleWritable, DoubleWritable, M>
+  public VertexReader<LongWritable, DoubleWritable, DoubleWritable>
   createVertexReader(InputSplit split, TaskAttemptContext context)
     throws IOException {
-    return new PseudoRandomVertexReader<M>();
+    return new PseudoRandomVertexReader();
   }
 
   /**
    * Used by {@link PseudoRandomVertexInputFormat} to read
    * pseudo-randomly generated data.
    */
-  private static class PseudoRandomVertexReader<M extends Writable> implements
-      VertexReader<LongWritable, DoubleWritable, DoubleWritable, M> {
+  private static class PseudoRandomVertexReader implements
+      VertexReader<LongWritable, DoubleWritable, DoubleWritable> {
     /** Logger. */
     private static final Logger LOG =
         Logger.getLogger(PseudoRandomVertexReader.class);
@@ -150,9 +147,9 @@ public class PseudoRandomVertexInputFormat<M extends Writable> extends
     }
 
     @Override
-    public Vertex<LongWritable, DoubleWritable, DoubleWritable, M>
+    public Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
     getCurrentVertex() throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable, DoubleWritable, M>
+      Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
       vertex = configuration.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/c5c1c4c6/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 1071196..3ffda1f 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
@@ -37,14 +37,12 @@ import java.util.List;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  * @param <X> Value type
  */
 @SuppressWarnings("rawtypes")
 public class SequenceFileVertexInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable,
-    X extends Vertex<I, V, E, M>>
-    extends VertexInputFormat<I, V, E, M> {
+    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 =
     new SequenceFileInputFormat<I, X>();
@@ -56,9 +54,9 @@ public class SequenceFileVertexInputFormat<I extends WritableComparable,
   }
 
   @Override
-  public VertexReader<I, V, E, M> createVertexReader(InputSplit split,
+  public VertexReader<I, V, E> createVertexReader(InputSplit split,
       TaskAttemptContext context) throws IOException {
-    return new SequenceFileVertexReader<I, V, E, M, X>(
+    return new SequenceFileVertexReader<I, V, E, X>(
         sequenceFileInputFormat.createRecordReader(split, context));
   }
 
@@ -68,13 +66,11 @@ public class SequenceFileVertexInputFormat<I extends WritableComparable,
    * @param <I> Vertex id
    * @param <V> Vertex data
    * @param <E> Edge data
-   * @param <M> Message data
    * @param <X> Value type
    */
   public static class SequenceFileVertexReader<I extends WritableComparable,
-      V extends Writable, E extends Writable, M extends Writable,
-      X extends Vertex<I, V, E, M>>
-      implements VertexReader<I, V, E, M> {
+      V extends Writable, E extends Writable, X extends Vertex<I, V, E, ?>>
+      implements VertexReader<I, V, E> {
     /** Internal record reader from {@link SequenceFileInputFormat} */
     private final RecordReader<I, X> recordReader;
 
@@ -97,7 +93,7 @@ public class SequenceFileVertexInputFormat<I extends WritableComparable,
       return recordReader.nextKeyValue();
     }
 
-    @Override public Vertex<I, V, E, M> getCurrentVertex()
+    @Override public Vertex<I, V, E, ?> getCurrentVertex()
       throws IOException, InterruptedException {
       return recordReader.getCurrentValue();
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 f7da40f..6e62b71 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
@@ -34,7 +34,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  */
 public class TextDoubleDoubleAdjacencyListVertexInputFormat<M extends Writable>
     extends AdjacencyListTextVertexInputFormat<Text, DoubleWritable,
-            DoubleWritable, M>  {
+            DoubleWritable>  {
 
   @Override
   public AdjacencyListTextVertexReader createVertexReader(InputSplit split,

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 898e57f..e47bda3 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
@@ -42,12 +42,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 abstract class TextVertexInputFormat<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable>
-    extends VertexInputFormat<I, V, E, M> {
+    V extends Writable, E extends Writable>
+    extends VertexInputFormat<I, V, E> {
   /** Uses the GiraphTextInputFormat to do everything */
   protected GiraphTextInputFormat textInputFormat = new GiraphTextInputFormat();
 
@@ -86,19 +85,19 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
    * {@link TextVertexReaderFromEachLineProcessedHandlingExceptions}.
    */
   protected abstract class TextVertexReader implements
-    VertexReader<I, V, E, M> {
+    VertexReader<I, V, E> {
     /** Internal line record reader */
     private RecordReader<LongWritable, Text> lineRecordReader;
     /** Context passed to initialize */
     private TaskAttemptContext context;
     /** Cached configuration */
-    private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+    private ImmutableClassesGiraphConfiguration<I, V, E, Writable> conf;
 
     @Override
     public void initialize(InputSplit inputSplit, TaskAttemptContext context)
       throws IOException, InterruptedException {
       this.context = context;
-      conf = new ImmutableClassesGiraphConfiguration<I, V, E, M>(
+      conf = new ImmutableClassesGiraphConfiguration<I, V, E, Writable>(
           context.getConfiguration());
       lineRecordReader = createLineRecordReader(inputSplit, context);
       lineRecordReader.initialize(inputSplit, context);
@@ -158,7 +157,7 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
      *
      * @return Configuration for this reader
      */
-    protected ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
+    protected ImmutableClassesGiraphConfiguration<I, V, E, Writable> getConf() {
       return conf;
     }
   }
@@ -171,10 +170,10 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
     TextVertexReader {
 
     @Override
-    public final Vertex<I, V, E, M> getCurrentVertex() throws IOException,
+    public final Vertex<I, V, E, ?> getCurrentVertex() throws IOException,
     InterruptedException {
       Text line = getRecordReader().getCurrentValue();
-      Vertex<I, V, E, M> vertex = getConf().createVertex();
+      Vertex<I, V, E, ?> vertex = getConf().createVertex();
       vertex.initialize(getId(line), getValue(line), getEdges(line));
       return vertex;
     }
@@ -239,10 +238,10 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
     }
 
     @Override
-    public final Vertex<I, V, E, M> getCurrentVertex() throws IOException,
+    public final Vertex<I, V, E, ?> getCurrentVertex() throws IOException,
     InterruptedException {
       Text line = getRecordReader().getCurrentValue();
-      Vertex<I, V, E, M> vertex;
+      Vertex<I, V, E, ?> vertex;
       T processed = preprocessLine(line);
       vertex = getConf().createVertex();
       vertex.initialize(getId(processed), getValue(processed),
@@ -323,11 +322,11 @@ public abstract class TextVertexInputFormat<I extends WritableComparable,
 
     @SuppressWarnings("unchecked")
     @Override
-    public final Vertex<I, V, E, M> 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, M> vertex;
+      Vertex<I, V, E, ?> vertex;
       T processed = null;
       try {
         processed = preprocessLine(line);
@@ -418,7 +417,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, M> 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/c5c1c4c6/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 e09f6a3..6d133ae 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
@@ -44,7 +44,7 @@ import java.util.List;
 @SuppressWarnings("rawtypes")
 public abstract class TextVertexValueInputFormat<I extends WritableComparable,
     V extends Writable, E extends Writable, M extends Writable>
-    extends VertexValueInputFormat<I, V, E, M> {
+    extends VertexValueInputFormat<I, V> {
   /** Uses the GiraphTextInputFormat to do everything */
   protected GiraphTextInputFormat textInputFormat = new GiraphTextInputFormat();
 
@@ -64,7 +64,7 @@ public abstract class TextVertexValueInputFormat<I extends WritableComparable,
    * {@link VertexValueReader} for {@link VertexValueInputFormat}.
    */
   protected abstract class TextVertexValueReader extends
-      VertexValueReader<I, V, E, M> {
+      VertexValueReader<I, V> {
     /** Internal line record reader */
     private RecordReader<LongWritable, Text> lineRecordReader;
     /** Context passed to initialize */

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 7493942..7e695ac 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
@@ -33,24 +33,22 @@ import java.io.IOException;
  * @param <I> Vertex id
  * @param <V> Vertex data
  * @param <E> Edge data
- * @param <M> Message data
  */
 public class VertexReaderWrapper<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    VertexReader<I, V, E, M> {
+    V extends Writable, E extends Writable> implements VertexReader<I, V, E> {
   /** Wrapped edge reader */
-  private GiraphReader<Vertex<I, V, E, M>> vertexReader;
+  private GiraphReader<Vertex<I, V, E, ?>> vertexReader;
   /** {@link VertexReader}-like wrapper of {@link #vertexReader} */
-  private IteratorToReaderWrapper<Vertex<I, V, E, M>> iterator;
+  private IteratorToReaderWrapper<Vertex<I, V, E, ?>> iterator;
 
   /**
    * Constructor
    *
    * @param vertexReader GiraphReader for vertices to wrap
    */
-  public VertexReaderWrapper(GiraphReader<Vertex<I, V, E, M>> vertexReader) {
+  public VertexReaderWrapper(GiraphReader<Vertex<I, V, E, ?>> vertexReader) {
     this.vertexReader = vertexReader;
-    iterator = new IteratorToReaderWrapper<Vertex<I, V, E, M>>(vertexReader);
+    iterator = new IteratorToReaderWrapper<Vertex<I, V, E, ?>>(vertexReader);
   }
 
   @Override
@@ -59,7 +57,7 @@ public class VertexReaderWrapper<I extends WritableComparable,
   }
 
   @Override
-  public Vertex<I, V, E, M> 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/c5c1c4c6/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 f55cf18..54b6bbd 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
@@ -217,7 +217,7 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
 
   /** Verify matching generic types in VertexInputFormat. */
   private void verifyVertexInputFormatGenericTypes() {
-    Class<? extends VertexInputFormat<I, V, E, M>> vertexInputFormatClass =
+    Class<? extends VertexInputFormat<I, V, E>> vertexInputFormatClass =
       conf.getVertexInputFormatClass();
     if (vertexInputFormatClass != null) {
       List<Class<?>> classList =

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 4559779..7d02533 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
@@ -686,7 +686,7 @@ public class BspServiceMaster<I extends WritableComparable,
     if (!getConfiguration().hasVertexInputFormat()) {
       return 0;
     }
-    VertexInputFormat<I, V, E, M> vertexInputFormat =
+    VertexInputFormat<I, V, E> vertexInputFormat =
         getConfiguration().createVertexInputFormat();
     return createInputSplits(vertexInputFormat, vertexInputSplitsPaths,
         "Vertex");

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 0ed9155..8e56eac 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
@@ -38,13 +38,11 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * @param <I> The Input
  * @param <V> The vertex type
  * @param <E> The edge type
- * @param <M> The message type
  */
 public class InMemoryVertexInputFormat<I extends WritableComparable,
                                  V extends Writable,
-                                 E extends Writable,
-                                 M extends Writable>
-    extends VertexInputFormat<I, V, E, M> {
+                                 E extends Writable>
+    extends VertexInputFormat<I, V, E> {
   /** The graph */
   private static TestGraph GRAPH;
 
@@ -69,13 +67,13 @@ public class InMemoryVertexInputFormat<I extends WritableComparable,
   }
 
   @Override
-  public VertexReader<I, V, E, M> createVertexReader(InputSplit inputSpplit,
+  public VertexReader<I, V, E> createVertexReader(InputSplit inputSpplit,
       TaskAttemptContext context) throws IOException {
 
-    return new VertexReader<I, V, E, M>() {
+    return new VertexReader<I, V, E>() {
       /** The iterator */
-      private Iterator<Vertex<I, V, E, M>> vertexIterator;
-      private Vertex<I, V, E, M> currentVertex;
+      private Iterator<Vertex<I, V, E, ?>> vertexIterator;
+      private Vertex<I, V, E, ?> currentVertex;
 
       @Override
       public void initialize(InputSplit inputSplit,
@@ -93,7 +91,7 @@ public class InMemoryVertexInputFormat<I extends WritableComparable,
       }
 
       @Override
-      public Vertex<I, V, E, M> getCurrentVertex() {
+      public Vertex<I, V, E, ?> getCurrentVertex() {
         return currentVertex;
       }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 570b1bc..c18648b 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
@@ -111,9 +111,9 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
       InputSplit inputSplit,
       GraphState<I, V, E, M> graphState)
     throws IOException, InterruptedException {
-    VertexInputFormat<I, V, E, M> vertexInputFormat =
+    VertexInputFormat<I, V, E> vertexInputFormat =
         configuration.createVertexInputFormat();
-    VertexReader<I, V, E, M> vertexReader =
+    VertexReader<I, V, E> vertexReader =
         vertexInputFormat.createVertexReader(inputSplit, context);
     vertexReader.initialize(inputSplit, context);
     long inputSplitVerticesLoaded = 0;
@@ -121,7 +121,7 @@ public class VertexInputSplitsCallable<I extends WritableComparable,
     long inputSplitEdgesLoaded = 0;
     while (vertexReader.nextVertex()) {
       Vertex<I, V, E, M> readerVertex =
-          vertexReader.getCurrentVertex();
+          (Vertex<I, V, E, M>) vertexReader.getCurrentVertex();
       if (readerVertex.getId() == null) {
         throw new IllegalArgumentException(
             "readInputSplit: Vertex reader returned a vertex " +

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 6a0b912..3414088 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
@@ -45,7 +45,7 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoubleDoubleAdjacencyListVertexInputFormat<BooleanWritable> {
+public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoubleDoubleAdjacencyListVertexInputFormat {
 
   private RecordReader<LongWritable, Text> rr;
   private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
@@ -132,7 +132,7 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
     vr.initialize(null, tac);
 
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<LongWritable, DoubleWritable, DoubleWritable, BooleanWritable>
+    Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
         vertex = vr.getCurrentVertex();
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex,
@@ -153,7 +153,7 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
 
     vr.initialize(null, tac);
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<LongWritable, DoubleWritable, DoubleWritable, BooleanWritable>
+    Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
         vertex = vr.getCurrentVertex();
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex,

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 6d81f51..7aea9ce 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
@@ -128,22 +128,22 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
   }
 
   public static <I extends WritableComparable, V extends Writable,
-      E extends WritableComparable, M extends Writable> void assertValidVertex(
-      ImmutableClassesGiraphConfiguration<I, V, E, M> conf,
-      GraphState<I, V, E, M> graphState,
-      Vertex<I, V, E, M> actual,
+      E extends WritableComparable> void assertValidVertex(
+      ImmutableClassesGiraphConfiguration<I, V, E, ?> conf,
+      GraphState<I, V, E, ?> graphState,
+      Vertex<I, V, E, ?> actual,
       I expectedId,
       V expectedValue,
       Edge<I, E>... edges) throws Exception {
-    Vertex<I, V, E, M> expected = conf.createVertex();
+    Vertex<I, V, E, ?> expected = conf.createVertex();
     setGraphState(expected, graphState);
     expected.initialize(expectedId, expectedValue, Arrays.asList(edges));
     assertValid(expected, actual);
   }
 
   public static <I extends WritableComparable, V extends Writable,
-      E extends WritableComparable, M extends Writable> void assertValid(
-      Vertex<I, V, E, M> expected, Vertex<I, V, E, M> actual) {
+      E extends WritableComparable> void assertValid(
+      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());
@@ -159,8 +159,7 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
 
     vr.initialize(null, tac);
     assertTrue("Should have been able to add a vertex", vr.nextVertex());
-    Vertex<Text, DoubleWritable, DoubleWritable, BooleanWritable> vertex =
-        vr.getCurrentVertex();
+    Vertex<Text, DoubleWritable, DoubleWritable, ?> vertex = vr.getCurrentVertex();
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex,
         new Text("Hi"), new DoubleWritable(0),
@@ -187,8 +186,7 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
 
     vr.initialize(null, tac);
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<Text, DoubleWritable, DoubleWritable, BooleanWritable> vertex =
-        vr.getCurrentVertex();
+    Vertex<Text, DoubleWritable, DoubleWritable, ?> vertex = vr.getCurrentVertex();
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex,
         new Text("BYE"), new DoubleWritable(0.01d),
@@ -209,8 +207,7 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
 
     vr.initialize(null, tac);
     assertTrue("Should have been able to read vertex", vr.nextVertex());
-    Vertex<Text, DoubleWritable, DoubleWritable, BooleanWritable> vertex =
-        vr.getCurrentVertex();
+    Vertex<Text, DoubleWritable, DoubleWritable, ?> vertex = vr.getCurrentVertex();
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex,
         new Text("alpha"), new DoubleWritable(42d),

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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 10bb4d0..8cd427e 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
@@ -22,7 +22,7 @@ import org.apache.giraph.conf.GiraphClasses;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.io.formats.IntNullNullNullTextInputFormat;
+import org.apache.giraph.io.formats.IntNullNullTextInputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -94,7 +94,7 @@ public class TestMasterObserver {
     GiraphClasses classes = new GiraphClasses();
     classes.setVertexClass(NoOpVertex.class);
     classes.setVertexEdgesClass(ByteArrayEdges.class);
-    classes.setVertexInputFormatClass(IntNullNullNullTextInputFormat.class);
+    classes.setVertexInputFormatClass(IntNullNullTextInputFormat.class);
     InternalVertexRunner.run(classes, params, graph);
 
     assertEquals(2, Obs.preApp);

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-examples/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java b/giraph-examples/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
index 26f970d..9e4cb83 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/GeneratedVertexReader.java
@@ -34,12 +34,11 @@ import java.io.IOException;
  * @param <I> Vertex index value
  * @param <V> Vertex value
  * @param <E> Edge value
- * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
 public abstract class GeneratedVertexReader<
-    I extends WritableComparable, V extends Writable, E extends Writable,
-    M extends Writable> implements VertexReader<I, V, E, M> {
+    I extends WritableComparable, V extends Writable,
+    E extends Writable> implements VertexReader<I, V, E> {
   /** Vertices produced by this reader */
   public static final String READER_VERTICES =
     "GeneratedVertexReader.reader_vertices";

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleDoubleTextInputFormat.java
deleted file mode 100644
index 6fef9f2..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleDoubleTextInputFormat.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 com.google.common.collect.Lists;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
-import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.io.formats.TextVertexInputFormat;
-import org.apache.giraph.edge.Edge;
-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.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.regex.Pattern;
-
-/**
- * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
- * unweighted graphs with long ids. Each line consists of: vertex neighbor1
- * neighbor2 ...
- */
-public class LongDoubleDoubleDoubleTextInputFormat
-    extends TextVertexInputFormat<LongWritable, DoubleWritable,
-    DoubleWritable, DoubleWritable>
-    implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
-    DoubleWritable, DoubleWritable> {
-  /** Configuration. */
-  private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, DoubleWritable> conf;
-
-  @Override
-  public TextVertexReader createVertexReader(InputSplit split,
-      TaskAttemptContext context)
-    throws IOException {
-    return new LongDoubleDoubleDoubleVertexReader();
-  }
-
-  @Override
-  public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
-      DoubleWritable, DoubleWritable, DoubleWritable> configuration) {
-    this.conf = configuration;
-  }
-
-  @Override
-  public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      DoubleWritable, DoubleWritable> getConf() {
-    return conf;
-  }
-
-  /**
-   * Vertex reader associated with
-   * {@link LongDoubleDoubleDoubleTextInputFormat}.
-   */
-  public class LongDoubleDoubleDoubleVertexReader extends
-    TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable,
-        DoubleWritable>.TextVertexReader {
-    /** Separator of the vertex and neighbors */
-    private final Pattern separator = Pattern.compile("[\t ]");
-
-    @Override
-    public Vertex<LongWritable, DoubleWritable, DoubleWritable, DoubleWritable>
-    getCurrentVertex() throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable, DoubleWritable, DoubleWritable>
-        vertex = conf.createVertex();
-
-      String[] tokens =
-          separator.split(getRecordReader().getCurrentValue().toString());
-      List<Edge<LongWritable, DoubleWritable>> edges =
-          Lists.newArrayListWithCapacity(tokens.length - 1);
-      float weight = 1.0f / (tokens.length - 1);
-      for (int n = 1; n < tokens.length; n++) {
-        edges.add(EdgeFactory.create(
-            new LongWritable(Long.parseLong(tokens[n])),
-            new DoubleWritable(weight)));
-      }
-
-      LongWritable vertexId = new LongWritable(Long.parseLong(tokens[0]));
-      vertex.initialize(vertexId, new DoubleWritable(), edges);
-
-      return vertex;
-    }
-
-    @Override
-    public boolean nextVertex() throws IOException, InterruptedException {
-      return getRecordReader().nextKeyValue();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/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
new file mode 100644
index 0000000..62bea5a
--- /dev/null
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.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 com.google.common.collect.Lists;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.giraph.edge.Edge;
+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;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
+/**
+ * Simple text-based {@link org.apache.giraph.io.VertexInputFormat} for
+ * unweighted graphs with long ids. Each line consists of: vertex neighbor1
+ * neighbor2 ...
+ */
+public class LongDoubleDoubleTextInputFormat
+    extends TextVertexInputFormat<LongWritable, DoubleWritable,
+    DoubleWritable>
+    implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
+    DoubleWritable, Writable> {
+  /** Configuration. */
+  private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
+      DoubleWritable, Writable> conf;
+
+  @Override
+  public TextVertexReader createVertexReader(InputSplit split,
+      TaskAttemptContext context)
+    throws IOException {
+    return new LongDoubleDoubleDoubleVertexReader();
+  }
+
+  @Override
+  public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
+      DoubleWritable, DoubleWritable, Writable> configuration) {
+    this.conf = configuration;
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
+      DoubleWritable, Writable> getConf() {
+    return conf;
+  }
+
+  /**
+   * Vertex reader associated with
+   * {@link LongDoubleDoubleTextInputFormat}.
+   */
+  public class LongDoubleDoubleDoubleVertexReader extends
+    TextVertexInputFormat<LongWritable, DoubleWritable,
+        DoubleWritable>.TextVertexReader {
+    /** Separator of the vertex and neighbors */
+    private final Pattern separator = Pattern.compile("[\t ]");
+
+    @Override
+    public Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+    getCurrentVertex() throws IOException, InterruptedException {
+      Vertex<LongWritable, DoubleWritable, DoubleWritable, ?>
+        vertex = conf.createVertex();
+
+      String[] tokens =
+          separator.split(getRecordReader().getCurrentValue().toString());
+      List<Edge<LongWritable, DoubleWritable>> edges =
+          Lists.newArrayListWithCapacity(tokens.length - 1);
+      float weight = 1.0f / (tokens.length - 1);
+      for (int n = 1; n < tokens.length; n++) {
+        edges.add(EdgeFactory.create(
+            new LongWritable(Long.parseLong(tokens[n])),
+            new DoubleWritable(weight)));
+      }
+
+      LongWritable vertexId = new LongWritable(Long.parseLong(tokens[0]));
+      vertex.initialize(vertexId, new DoubleWritable(), edges);
+
+      return vertex;
+    }
+
+    @Override
+    public boolean nextVertex() throws IOException, InterruptedException {
+      return getRecordReader().nextKeyValue();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/c5c1c4c6/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullDoubleTextInputFormat.java
deleted file mode 100644
index e22194a..0000000
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullDoubleTextInputFormat.java
+++ /dev/null
@@ -1,106 +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.conf.ImmutableClassesGiraphConfigurable;
-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.TextVertexInputFormat;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.regex.Pattern;
-
-/**
- * Input format for unweighted graphs with long ids and double vertex values
- */
-public class LongDoubleNullDoubleTextInputFormat
-    extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable,
-    DoubleWritable>
-    implements ImmutableClassesGiraphConfigurable<LongWritable, DoubleWritable,
-    NullWritable, DoubleWritable> {
-  /** Configuration. */
-  private ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      NullWritable, DoubleWritable> conf;
-
-  @Override
-  public TextVertexReader createVertexReader(InputSplit split,
-                                             TaskAttemptContext context)
-    throws IOException {
-    return new LongDoubleNullDoubleVertexReader();
-  }
-
-  @Override
-  public void setConf(ImmutableClassesGiraphConfiguration<LongWritable,
-      DoubleWritable, NullWritable, DoubleWritable> configuration) {
-    this.conf = configuration;
-  }
-
-  @Override
-  public ImmutableClassesGiraphConfiguration<LongWritable, DoubleWritable,
-      NullWritable, DoubleWritable> getConf() {
-    return conf;
-  }
-
-  /**
-   * Vertex reader associated with
-   * {@link LongDoubleNullDoubleTextInputFormat}.
-   */
-  public class LongDoubleNullDoubleVertexReader extends
-      TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable,
-          DoubleWritable>.TextVertexReader {
-    /** Separator of the vertex and neighbors */
-    private final Pattern separator = Pattern.compile("[\t ]");
-
-    @Override
-    public Vertex<LongWritable, DoubleWritable, NullWritable, DoubleWritable>
-    getCurrentVertex() throws IOException, InterruptedException {
-      Vertex<LongWritable, DoubleWritable, NullWritable, DoubleWritable>
-          vertex = conf.createVertex();
-
-      String[] tokens =
-          separator.split(getRecordReader().getCurrentValue().toString());
-      List<Edge<LongWritable, NullWritable>> edges =
-          Lists.newArrayListWithCapacity(tokens.length - 1);
-      for (int n = 1; n < tokens.length; n++) {
-        edges.add(EdgeFactory.create(
-            new LongWritable(Long.parseLong(tokens[n])),
-            NullWritable.get()));
-      }
-
-      LongWritable vertexId = new LongWritable(Long.parseLong(tokens[0]));
-      vertex.initialize(vertexId, new DoubleWritable(), edges);
-
-      return vertex;
-    }
-
-    @Override
-    public boolean nextVertex() throws IOException, InterruptedException {
-      return getRecordReader().nextKeyValue();
-    }
-  }
-}