You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ni...@apache.org on 2013/02/17 18:01:29 UTC

git commit: GIRAPH-519: EdgeFactory (nitay)

Updated Branches:
  refs/heads/trunk 212326153 -> 028b0c92f


GIRAPH-519: EdgeFactory (nitay)


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

Branch: refs/heads/trunk
Commit: 028b0c92fde92e131d79e0140a01c13044995f17
Parents: 2123261
Author: Nitay Joffe <ni...@apache.org>
Authored: Sat Feb 16 23:29:09 2013 -0500
Committer: Nitay Joffe <ni...@apache.org>
Committed: Sun Feb 17 11:54:56 2013 -0500

----------------------------------------------------------------------
 CHANGELOG                                          |    2 +
 .../edgemarker/AccumuloEdgeInputFormat.java        |    4 +-
 .../conf/ImmutableClassesGiraphConfiguration.java  |   11 +-
 .../java/org/apache/giraph/graph/DefaultEdge.java  |    4 +-
 .../java/org/apache/giraph/graph/EdgeFactory.java  |   89 +++++++++++++++
 .../java/org/apache/giraph/graph/EdgeNoValue.java  |    9 +-
 .../org/apache/giraph/graph/VertexMutations.java   |   13 +--
 .../io/formats/IntIntNullIntTextInputFormat.java   |    4 +-
 .../io/formats/IntNullTextEdgeInputFormat.java     |    3 +-
 .../io/formats/JsonBase64VertexInputFormat.java    |    4 +-
 ...JsonLongDoubleFloatDoubleVertexInputFormat.java |    5 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |    5 +-
 .../io/formats/PseudoRandomEdgeInputFormat.java    |    4 +-
 .../io/formats/PseudoRandomVertexInputFormat.java  |    6 +-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |    4 +-
 .../giraph/io/formats/TextEdgeInputFormat.java     |   10 +-
 .../org/apache/giraph/utils/EdgeIterables.java     |    7 +-
 .../UnmodifiableLongFloatEdgeArrayIterable.java    |    6 +-
 .../UnmodifiableLongNullEdgeArrayIterable.java     |    4 +-
 .../org/apache/giraph/utils/WritableUtils.java     |    6 +-
 .../apache/giraph/vertex/EdgeListVertexBase.java   |   10 +-
 .../org/apache/giraph/vertex/HashMapVertex.java    |    4 +-
 .../giraph/vertex/LongDoubleFloatDoubleVertex.java |    5 +-
 .../apache/giraph/vertex/SimpleMutableVertex.java  |    6 +-
 .../main/java/org/apache/giraph/vertex/Vertex.java |    4 +-
 .../java/org/apache/giraph/comm/RequestTest.java   |    5 +-
 .../TestAdjacencyListTextVertexOutputFormat.java   |   14 +--
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |   15 ++-
 ...DoubleDoubleAdjacencyListVertexInputFormat.java |   16 ++--
 .../partition/TestGiraphTransferRegulator.java     |   13 +--
 .../giraph/vertex/TestIntIntNullIntVertex.java     |    6 +-
 .../apache/giraph/vertex/TestMultiGraphVertex.java |   32 ++----
 .../apache/giraph/vertex/TestMutableVertex.java    |   18 ++--
 .../LongDoubleFloatDoubleTextInputFormat.java      |    4 +-
 ...lizingLongDoubleFloatDoubleTextInputFormat.java |    5 +-
 .../giraph/examples/SimpleCheckpointVertex.java    |    7 +-
 .../giraph/examples/SimpleMutateGraphVertex.java   |    5 +-
 .../giraph/examples/SimplePageRankVertex.java      |    5 +-
 .../giraph/examples/SimpleSuperstepVertex.java     |    5 +-
 .../org/apache/giraph/examples/VerifyMessage.java  |    6 +-
 .../examples/SimpleShortestPathsVertexTest.java    |   23 ++--
 .../examples/SimpleTriangleClosingVertexTest.java  |    6 +-
 .../io/hbase/edgemarker/TableEdgeInputFormat.java  |    4 +-
 .../io/hcatalog/HCatalogEdgeInputFormat.java       |    7 +-
 .../io/hcatalog/HCatalogVertexInputFormat.java     |   10 +-
 45 files changed, 243 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index ee31207..776ace2 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 0.2.0 - unreleased
+  GIRAPH-519: EdgeFactory (nitay)
+
   GIRAPH-515: More efficient and flexible edge-based input (apresta)
 
   GIRAPH-516: out-of-core messages dies for ArrayIndexOutOfBoundsException when 

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 8586dae..30d40f6 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
@@ -19,8 +19,8 @@ package org.apache.giraph.io.accumulo.edgemarker;
 
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.io.accumulo.AccumuloVertexInputFormat;
 import org.apache.giraph.vertex.Vertex;
@@ -87,7 +87,7 @@ public class AccumuloEdgeInputFormat
               List<Edge<Text, Text>> edges = Lists.newLinkedList();
               String edge = new String(value.get());
               Text edgeId = new Text(edge);
-              edges.add(new DefaultEdge<Text, Text>(edgeId, uselessEdgeValue));
+              edges.add(EdgeFactory.create(edgeId, uselessEdgeValue));
               vertex.initialize(vertexId, new Text(), edges);
 
             return vertex;

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 18fd9ef..e6c4cc6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
@@ -20,9 +20,8 @@ package org.apache.giraph.conf;
 
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.MutableEdge;
 import org.apache.giraph.graph.VertexResolver;
@@ -509,9 +508,9 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    */
   public Edge<I, E> createEdge() {
     if (isEdgeValueNullWritable()) {
-      return (Edge<I, E>) new EdgeNoValue<I>(createVertexId());
+      return (Edge<I, E>) EdgeFactory.create(createVertexId());
     } else {
-      return new DefaultEdge<I, E>(createVertexId(), createEdgeValue());
+      return EdgeFactory.create(createVertexId(), createEdgeValue());
     }
   }
 
@@ -522,9 +521,9 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    */
   public MutableEdge<I, E> createMutableEdge() {
     if (isEdgeValueNullWritable()) {
-      return (MutableEdge<I, E>) new EdgeNoValue<I>(createVertexId());
+      return (MutableEdge<I, E>) EdgeFactory.createMutable(createVertexId());
     } else {
-      return new DefaultEdge<I, E>(createVertexId(), createEdgeValue());
+      return EdgeFactory.createMutable(createVertexId(), createEdgeValue());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/graph/DefaultEdge.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultEdge.java b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultEdge.java
index 347abbf..039f0d7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultEdge.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultEdge.java
@@ -44,12 +44,12 @@ public class DefaultEdge<I extends WritableComparable, E extends Writable>
   public DefaultEdge() { }
 
   /**
-   * Create the edge with final values
+   * Create the edge with final values. Don't call, use EdgeFactory instead.
    *
    * @param targetVertexId Desination vertex id.
    * @param value Value of the edge.
    */
-  public DefaultEdge(I targetVertexId, E value) {
+  DefaultEdge(I targetVertexId, E value) {
     this.targetVertexId = targetVertexId;
     this.value = value;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/graph/EdgeFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/EdgeFactory.java b/giraph-core/src/main/java/org/apache/giraph/graph/EdgeFactory.java
new file mode 100644
index 0000000..a3e6efb
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/EdgeFactory.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.graph;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Factory for creating Edges
+ */
+public class EdgeFactory {
+  /** Do not construct */
+  private EdgeFactory() { }
+
+  /**
+   * Create an edge pointing to a given ID with a value
+   *
+   * @param id target ID
+   * @param value edge value
+   * @param <I> Vertex ID type
+   * @param <E> Edge Value type
+   * @return Edge pointing to ID with value
+   */
+  public static <I extends WritableComparable,
+                 E extends Writable>
+  Edge<I, E> create(I id, E value) {
+    return createMutable(id, value);
+  }
+
+  /**
+   * Create an edge pointing to a given ID without a value
+   *
+   * @param id target ID
+   * @param <I> Vertex ID type
+   * @return Edge pointing to ID without a value
+   */
+  public static <I extends WritableComparable>
+  Edge<I, NullWritable> create(I id) {
+    return createMutable(id);
+  }
+
+  /**
+   * Create a mutable edge pointing to a given ID with a value
+   *
+   * @param id target ID
+   * @param value edge value
+   * @param <I> Vertex ID type
+   * @param <E> Edge Value type
+   * @return Edge pointing to ID with value
+   */
+  public static <I extends WritableComparable,
+                 E extends Writable>
+  MutableEdge<I, E> createMutable(I id, E value) {
+    if (value instanceof NullWritable) {
+      return (MutableEdge<I, E>) createMutable(id);
+    } else {
+      return new DefaultEdge<I, E>(id, value);
+    }
+  }
+
+  /**
+   * Create a mutable edge pointing to a given ID with a value
+   *
+   * @param id target ID
+   * @param <I> Vertex ID type
+   * @return Edge pointing to ID with value
+   */
+  public static <I extends WritableComparable>
+  MutableEdge<I, NullWritable> createMutable(I id) {
+    return new EdgeNoValue<I>(id);
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/graph/EdgeNoValue.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/EdgeNoValue.java b/giraph-core/src/main/java/org/apache/giraph/graph/EdgeNoValue.java
index 765e602..4ac6759 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/EdgeNoValue.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/EdgeNoValue.java
@@ -34,15 +34,14 @@ public class EdgeNoValue<I extends WritableComparable>
   private I targetVertexId = null;
 
   /** Empty constructor */
-  public EdgeNoValue() {
-    // do nothing
-  }
+  EdgeNoValue() { }
 
   /**
-   * Constructor with target vertex ID
+   * Constructor with target vertex ID. Don't call, use EdgeFactory instead.
+   *
    * @param targetVertexId vertex ID
    */
-  public EdgeNoValue(I targetVertexId) {
+  EdgeNoValue(I targetVertexId) {
     this.targetVertexId = targetVertexId;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java b/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
index 431d1cf..fa33341 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
@@ -20,6 +20,7 @@ package org.apache.giraph.graph;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.WritableUtils;
 import org.apache.giraph.vertex.Vertex;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -93,15 +94,9 @@ public class VertexMutations<I extends WritableComparable,
     removedVertexCount = input.readInt();
     int addedEdgeListSize = input.readInt();
     for (int i = 0; i < addedEdgeListSize; ++i) {
-      I destVertex = conf.createVertexId();
-      destVertex.readFields(input);
-      if (conf.isEdgeValueNullWritable()) {
-        addedEdgeList.add((Edge<I, E>) new EdgeNoValue<I>(destVertex));
-      } else {
-        E edgeValue = conf.createEdgeValue();
-        edgeValue.readFields(input);
-        addedEdgeList.add(new DefaultEdge<I, E>(destVertex, edgeValue));
-      }
+      Edge<I, E> edge = conf.createEdge();
+      WritableUtils.readEdge(input, edge);
+      addedEdgeList.add(edge);
     }
     int removedEdgeListSize = input.readInt();
     for (int i = 0; i < removedEdgeListSize; ++i) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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
index d4d9e61..b00e495 100644
--- 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
@@ -19,7 +19,7 @@
 package org.apache.giraph.io.formats;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
@@ -84,7 +84,7 @@ public class IntIntNullIntTextInputFormat extends
       List<Edge<IntWritable, NullWritable>> edges =
           Lists.newArrayListWithCapacity(tokens.length - 1);
       for (int n = 1; n < tokens.length; n++) {
-        edges.add(new EdgeNoValue<IntWritable>(
+        edges.add(EdgeFactory.create(
             new IntWritable(Integer.parseInt(tokens[n]))));
       }
       return edges;

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullTextEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullTextEdgeInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullTextEdgeInputFormat.java
index 108f96c..2b4b5d5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullTextEdgeInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/IntNullTextEdgeInputFormat.java
@@ -18,6 +18,7 @@
 
 package org.apache.giraph.io.formats;
 
+import org.apache.giraph.io.EdgeReader;
 import org.apache.giraph.utils.IntPair;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -40,7 +41,7 @@ public class IntNullTextEdgeInputFormat extends
   private static final Pattern SEPARATOR = Pattern.compile("[\t ]");
 
   @Override
-  public TextEdgeReader createEdgeReader(
+  public EdgeReader<IntWritable, NullWritable> createEdgeReader(
       InputSplit split, TaskAttemptContext context) throws IOException {
     return new IntNullTextEdgeReader();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 34e6059..21ca427 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
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.io.formats;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -142,7 +142,7 @@ public class JsonBase64VertexInputFormat<I extends WritableComparable,
         targetVertexId.readFields(input);
         E edgeValue = getConf().createEdgeValue();
         edgeValue.readFields(input);
-        edges.add(new DefaultEdge<I, E>(targetVertexId, edgeValue));
+        edges.add(EdgeFactory.create(targetVertexId, edgeValue));
       }
       return edges;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 3a8d5de..2df20f1 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
@@ -17,8 +17,8 @@
  */
 package org.apache.giraph.io.formats;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.vertex.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -90,8 +90,7 @@ public class JsonLongDoubleFloatDoubleVertexInputFormat extends
           Lists.newArrayListWithCapacity(jsonEdgeArray.length());
       for (int i = 0; i < jsonEdgeArray.length(); ++i) {
         JSONArray jsonEdge = jsonEdgeArray.getJSONArray(i);
-        edges.add(new DefaultEdge<LongWritable, FloatWritable>(
-            new LongWritable(jsonEdge.getLong(0)),
+        edges.add(EdgeFactory.create(new LongWritable(jsonEdge.getLong(0)),
             new FloatWritable((float) jsonEdge.getDouble(1))));
       }
       return edges;

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 7463777..4e35201 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
@@ -17,8 +17,8 @@
  */
 package org.apache.giraph.io.formats;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Writable;
@@ -74,8 +74,7 @@ public class LongDoubleDoubleAdjacencyListVertexInputFormat<M extends Writable>
 
     @Override
     public Edge<LongWritable, DoubleWritable> decodeEdge(String s1, String s2) {
-      return new DefaultEdge<LongWritable, DoubleWritable>(
-          new LongWritable(Long.valueOf(s1)),
+      return EdgeFactory.create(new LongWritable(Long.valueOf(s1)),
           new DoubleWritable(Double.valueOf(s2)));
     }
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
index 116f45e..90f814c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomEdgeInputFormat.java
@@ -20,8 +20,8 @@ package org.apache.giraph.io.formats;
 
 import org.apache.giraph.bsp.BspInputSplit;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.EdgeReader;
 import org.apache.hadoop.io.DoubleWritable;
@@ -184,7 +184,7 @@ public class PseudoRandomEdgeInputFormat
         LOG.trace("getCurrentEdge: Return edge (" + currentVertexId + ", " +
             "" + destVertexId + ")");
       }
-      return new DefaultEdge<LongWritable, DoubleWritable>(
+      return EdgeFactory.create(
               destVertexId,
               new DoubleWritable(random.nextDouble()));
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 7aae427..f2a2c93 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
@@ -20,8 +20,8 @@ package org.apache.giraph.io.formats;
 
 import org.apache.giraph.bsp.BspInputSplit;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.vertex.Vertex;
@@ -172,8 +172,8 @@ public class PseudoRandomVertexInputFormat<M extends Writable> extends
             new LongWritable(Math.abs(rand.nextLong()) %
               aggregateVertices);
         } while (destVertices.contains(destVertexId));
-        edges.add(new DefaultEdge<LongWritable, DoubleWritable>(
-            destVertexId, new DoubleWritable(rand.nextDouble())));
+        edges.add(EdgeFactory.create(destVertexId,
+            new DoubleWritable(rand.nextDouble())));
         destVertices.add(destVertexId);
       }
       vertex.initialize(new LongWritable(vertexId), vertexValue, edges);

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 14862f7..36d00db 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
@@ -17,8 +17,8 @@
  */
 package org.apache.giraph.io.formats;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -71,7 +71,7 @@ public class TextDoubleDoubleAdjacencyListVertexInputFormat<M extends Writable>
 
     @Override
     public Edge<Text, DoubleWritable> decodeEdge(String s1, String s2) {
-      return new DefaultEdge<Text, DoubleWritable>(new Text(s1),
+      return EdgeFactory.create(new Text(s1),
           new DoubleWritable(Double.valueOf(s2)));
     }
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/io/formats/TextEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextEdgeInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextEdgeInputFormat.java
index a8ebfda..6e59e7f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextEdgeInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextEdgeInputFormat.java
@@ -19,8 +19,8 @@
 package org.apache.giraph.io.formats;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.EdgeReader;
 import org.apache.hadoop.io.LongWritable;
@@ -58,10 +58,6 @@ public abstract class TextEdgeInputFormat<I extends WritableComparable,
     return textInputFormat.getEdgeSplits(context);
   }
 
-  @Override
-  public abstract TextEdgeReader createEdgeReader(
-      InputSplit split, TaskAttemptContext context) throws IOException;
-
   /**
    * {@link EdgeReader} for {@link TextEdgeInputFormat}.
    */
@@ -164,7 +160,7 @@ public abstract class TextEdgeInputFormat<I extends WritableComparable,
       Text line = getRecordReader().getCurrentValue();
       I targetVertexId = getTargetVertexId(line);
       E edgeValue = getValue(line);
-      return new DefaultEdge<I, E>(targetVertexId, edgeValue);
+      return EdgeFactory.create(targetVertexId, edgeValue);
     }
 
     @Override
@@ -234,7 +230,7 @@ public abstract class TextEdgeInputFormat<I extends WritableComparable,
       T processed = processCurrentLine();
       I targetVertexId = getTargetVertexId(processed);
       E edgeValue = getValue(processed);
-      return new DefaultEdge<I, E>(targetVertexId, edgeValue);
+      return EdgeFactory.create(targetVertexId, edgeValue);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/utils/EdgeIterables.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/EdgeIterables.java b/giraph-core/src/main/java/org/apache/giraph/utils/EdgeIterables.java
index 51f7409..ab288fb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/EdgeIterables.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/EdgeIterables.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.utils;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -61,14 +61,13 @@ public class EdgeIterables {
    * @param <I> Vertex id type.
    * @return Edge iterable.
    */
-  public static
-  <I extends WritableComparable>
+  public static <I extends WritableComparable>
   Iterable<Edge<I, NullWritable>> getEdges(Iterable<I> neighbors) {
     return Iterables.transform(neighbors,
         new Function<I, Edge<I, NullWritable>>() {
           @Override
           public Edge<I, NullWritable> apply(I neighbor) {
-            return new EdgeNoValue<I>(neighbor);
+            return EdgeFactory.create(neighbor);
           }
         });
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongFloatEdgeArrayIterable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongFloatEdgeArrayIterable.java b/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongFloatEdgeArrayIterable.java
index fca4d45..b767058 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongFloatEdgeArrayIterable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongFloatEdgeArrayIterable.java
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.utils;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
 
@@ -60,8 +60,8 @@ public class UnmodifiableLongFloatEdgeArrayIterable extends
   @Override
   public Edge<LongWritable, FloatWritable> next() {
     Edge<LongWritable, FloatWritable> retval =
-        new DefaultEdge<LongWritable, FloatWritable>(new LongWritable(
-            longArray[offset]), new FloatWritable(floatArray[offset]));
+        EdgeFactory.create(new LongWritable(longArray[offset]),
+            new FloatWritable(floatArray[offset]));
     offset++;
     return retval;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongNullEdgeArrayIterable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongNullEdgeArrayIterable.java b/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongNullEdgeArrayIterable.java
index 5cc940c..18f280a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongNullEdgeArrayIterable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/UnmodifiableLongNullEdgeArrayIterable.java
@@ -18,7 +18,7 @@
 package org.apache.giraph.utils;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
 
@@ -60,7 +60,7 @@ public class UnmodifiableLongNullEdgeArrayIterable extends
   @Override
   public Edge<LongWritable, NullWritable> next() {
     Edge<LongWritable, NullWritable> retval =
-        new EdgeNoValue<LongWritable>(new LongWritable(longArray[offset]));
+        EdgeFactory.create(new LongWritable(longArray[offset]));
     offset++;
     return retval;
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index ef94645..129923d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -337,8 +337,7 @@ public class WritableUtils {
    * @throws IOException
    */
   public static <I extends WritableComparable, E extends Writable>
-  void writeEdge(DataOutput out, Edge<I, E> edge)
-    throws IOException {
+  void writeEdge(DataOutput out, Edge<I, E> edge) throws IOException {
     edge.getTargetVertexId().write(out);
     edge.getValue().write(out);
   }
@@ -353,8 +352,7 @@ public class WritableUtils {
    * @throws IOException
    */
   public static <I extends WritableComparable, E extends Writable>
-  void readEdge(DataInput in, Edge<I, E> edge)
-    throws IOException {
+  void readEdge(DataInput in, Edge<I, E> edge) throws IOException {
     edge.getTargetVertexId().readFields(in);
     edge.getValue().readFields(in);
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/vertex/EdgeListVertexBase.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/vertex/EdgeListVertexBase.java b/giraph-core/src/main/java/org/apache/giraph/vertex/EdgeListVertexBase.java
index 6f3918c..ec04569 100644
--- a/giraph-core/src/main/java/org/apache/giraph/vertex/EdgeListVertexBase.java
+++ b/giraph-core/src/main/java/org/apache/giraph/vertex/EdgeListVertexBase.java
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.vertex;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.utils.WritableUtils;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
@@ -85,11 +85,9 @@ public abstract class EdgeListVertexBase<I extends WritableComparable,
     int numEdges = in.readInt();
     edgeList = Lists.newArrayListWithCapacity(numEdges);
     for (int i = 0; i < numEdges; ++i) {
-      I targetVertexId = getConf().createVertexId();
-      targetVertexId.readFields(in);
-      E edgeValue = getConf().createEdgeValue();
-      edgeValue.readFields(in);
-      edgeList.add(new DefaultEdge<I, E>(targetVertexId, edgeValue));
+      Edge<I, E> edge = getConf().createEdge();
+      WritableUtils.readEdge(in, edge);
+      edgeList.add(edge);
     }
 
     readHaltBoolean(in);

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/vertex/HashMapVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/vertex/HashMapVertex.java b/giraph-core/src/main/java/org/apache/giraph/vertex/HashMapVertex.java
index 6bf266e..2160c3b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/vertex/HashMapVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/vertex/HashMapVertex.java
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.vertex;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
@@ -97,7 +97,7 @@ public abstract class HashMapVertex<I extends WritableComparable,
 
           @Override
           public Edge<I, E> apply(Map.Entry<I, E> edge) {
-            return new DefaultEdge<I, E>(edge.getKey(), edge.getValue());
+            return EdgeFactory.create(edge.getKey(), edge.getValue());
           }
         });
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/vertex/LongDoubleFloatDoubleVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/vertex/LongDoubleFloatDoubleVertex.java b/giraph-core/src/main/java/org/apache/giraph/vertex/LongDoubleFloatDoubleVertex.java
index 628b215..befadea 100644
--- a/giraph-core/src/main/java/org/apache/giraph/vertex/LongDoubleFloatDoubleVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/vertex/LongDoubleFloatDoubleVertex.java
@@ -17,8 +17,8 @@
  */
 package org.apache.giraph.vertex;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -102,8 +102,7 @@ public abstract class LongDoubleFloatDoubleVertex
           @Override
           public Edge<LongWritable, FloatWritable> next() {
             long targetVertex = targetVertices[offset++];
-            return new DefaultEdge<LongWritable, FloatWritable>(
-                new LongWritable(targetVertex),
+            return EdgeFactory.create(new LongWritable(targetVertex),
                 new FloatWritable(targetVertex));
           }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/vertex/SimpleMutableVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/vertex/SimpleMutableVertex.java b/giraph-core/src/main/java/org/apache/giraph/vertex/SimpleMutableVertex.java
index c5734c4..e12f783 100644
--- a/giraph-core/src/main/java/org/apache/giraph/vertex/SimpleMutableVertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/vertex/SimpleMutableVertex.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.vertex;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.utils.EdgeIterables;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
@@ -93,7 +93,7 @@ public abstract class SimpleMutableVertex<I extends WritableComparable,
    */
   public void addEdgeRequest(I sourceVertexId) throws IOException {
     getGraphState().getWorkerClientRequestProcessor().
-        addEdgeRequest(sourceVertexId, new EdgeNoValue<I>(sourceVertexId));
+        addEdgeRequest(sourceVertexId, EdgeFactory.create(sourceVertexId));
   }
 
   @Override
@@ -109,7 +109,7 @@ public abstract class SimpleMutableVertex<I extends WritableComparable,
     for (int i = 0; i < numEdges; ++i) {
       I targetVertexId = getConf().createVertexId();
       targetVertexId.readFields(in);
-      edges.add(new EdgeNoValue<I>(targetVertexId));
+      edges.add(EdgeFactory.create(targetVertexId));
     }
 
     initialize(vertexId, vertexValue, edges);

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/main/java/org/apache/giraph/vertex/Vertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/vertex/Vertex.java b/giraph-core/src/main/java/org/apache/giraph/vertex/Vertex.java
index db6dca3..a959ef8 100644
--- a/giraph-core/src/main/java/org/apache/giraph/vertex/Vertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/vertex/Vertex.java
@@ -20,8 +20,8 @@ package org.apache.giraph.vertex;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.partition.PartitionOwner;
@@ -372,7 +372,7 @@ public abstract class Vertex<I extends WritableComparable,
       targetVertexId.readFields(in);
       E edgeValue = (E) getConf().createEdgeValue();
       edgeValue.readFields(in);
-      edges.add(new DefaultEdge<I, E>(targetVertexId, edgeValue));
+      edges.add(EdgeFactory.create(targetVertexId, edgeValue));
     }
 
     initialize(vertexId, vertexValue, edges);

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
index ef1920d..7468c47 100644
--- a/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
+++ b/giraph-core/src/test/java/org/apache/giraph/comm/RequestTest.java
@@ -27,8 +27,8 @@ import org.apache.giraph.comm.requests.SendWorkerMessagesRequest;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.graph.VertexMutations;
 import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionStore;
@@ -220,8 +220,7 @@ public class RequestTest {
       }
       for (int j = 0; j < 5; ++j) {
         Edge<IntWritable, IntWritable> edge =
-            new DefaultEdge<IntWritable, IntWritable>(
-                new IntWritable(i), new IntWritable(2*j));
+            EdgeFactory.create(new IntWritable(i), new IntWritable(2 * j));
         mutations.addEdge(edge);
       }
       for (int j = 0; j < 7; ++j) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
index 3a258d9..036924f 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
@@ -17,8 +17,8 @@
  */
 package org.apache.giraph.io;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.formats.AdjacencyListTextVertexOutputFormat;
 import org.apache.giraph.vertex.Vertex;
 import org.apache.hadoop.conf.Configuration;
@@ -91,10 +91,8 @@ public class TestAdjacencyListTextVertexOutputFormat extends AdjacencyListTextVe
     when(vertex.getTotalNumEdges()).thenReturn(2l);
     List<Edge<Text, DoubleWritable>> cities = Lists.newArrayList();
     Collections.addAll(cities,
-        new DefaultEdge<Text, DoubleWritable>(
-            new Text("Los Angeles"), new DoubleWritable(347.16)),
-        new DefaultEdge<Text, DoubleWritable>(
-            new Text("Phoenix"), new DoubleWritable(652.48)));
+        EdgeFactory.create(new Text("Los Angeles"), new DoubleWritable(347.16)),
+        EdgeFactory.create(new Text("Phoenix"), new DoubleWritable(652.48)));
 
     when(vertex.getEdges()).thenReturn(cities);
 
@@ -122,10 +120,8 @@ public class TestAdjacencyListTextVertexOutputFormat extends AdjacencyListTextVe
     when(vertex.getTotalNumEdges()).thenReturn(2l);
     List<Edge<Text, DoubleWritable>> cities = Lists.newArrayList();
     Collections.addAll(cities,
-        new DefaultEdge<Text, DoubleWritable>(
-            new Text("Los Angeles"), new DoubleWritable(347.16)),
-        new DefaultEdge<Text, DoubleWritable>(
-            new Text("Phoenix"), new DoubleWritable(652.48)));
+        EdgeFactory.create(new Text("Los Angeles"), new DoubleWritable(347.16)),
+        EdgeFactory.create(new Text("Phoenix"), new DoubleWritable(652.48)));
 
     when(vertex.getEdges()).thenReturn(cities);
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 824e09e..7ef8eae 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
@@ -19,11 +19,11 @@ package org.apache.giraph.io;
 
 
 import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.graph.DefaultEdge;
+import org.apache.giraph.graph.EdgeFactory;
+import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.io.formats.AdjacencyListTextVertexInputFormat;
 import org.apache.giraph.io.formats.LongDoubleDoubleAdjacencyListVertexInputFormat;
 import org.apache.giraph.vertex.EdgeListVertex;
-import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.vertex.Vertex;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.BooleanWritable;
@@ -138,9 +138,9 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex,
         new LongWritable(42), new DoubleWritable(0.1),
-        new DefaultEdge<LongWritable, DoubleWritable>(new LongWritable(99), new DoubleWritable(0.2)),
-        new DefaultEdge<LongWritable, DoubleWritable>(new LongWritable(2000), new DoubleWritable(0.3)),
-        new DefaultEdge<LongWritable, DoubleWritable>(new LongWritable(4000), new DoubleWritable(0.4)));
+        EdgeFactory.create(new LongWritable(99), new DoubleWritable(0.2)),
+        EdgeFactory.create(new LongWritable(2000), new DoubleWritable(0.3)),
+        EdgeFactory.create(new LongWritable(4000), new DoubleWritable(0.4)));
     assertEquals(vertex.getNumEdges(), 3);
   }
 
@@ -157,8 +157,9 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
     Vertex<LongWritable, DoubleWritable, DoubleWritable, BooleanWritable>
         vertex = vr.getCurrentVertex();
     setGraphState(vertex, graphState);
-    assertValidVertex(conf, graphState, vertex, new LongWritable(12345), new DoubleWritable(42.42),
-       new DefaultEdge<LongWritable, DoubleWritable>(new LongWritable(9999999), new DoubleWritable(99.9)));
+    assertValidVertex(conf, graphState, vertex,
+        new LongWritable(12345), new DoubleWritable(42.42),
+       EdgeFactory.create(new LongWritable(9999999), new DoubleWritable(99.9)));
     assertEquals(vertex.getNumEdges(), 1);
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 b08e7ad..d1e267e 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
@@ -19,8 +19,8 @@ package org.apache.giraph.io;
 
 import org.apache.giraph.bsp.BspUtils;
 import org.apache.giraph.conf.GiraphConstants;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.io.formats.AdjacencyListTextVertexInputFormat;
 import org.apache.giraph.io.formats.TextDoubleDoubleAdjacencyListVertexInputFormat;
@@ -173,9 +173,9 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
         vr.getCurrentVertex();
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex, new Text("Hi"), new DoubleWritable(0),
-        new DefaultEdge<Text, DoubleWritable>(new Text("Ciao"), new DoubleWritable(1.123d)),
-        new DefaultEdge<Text, DoubleWritable>(new Text("Bomdia"), new DoubleWritable(2.234d)),
-        new DefaultEdge<Text, DoubleWritable>(new Text("Ola"), new DoubleWritable(3.345d)));
+        EdgeFactory.create(new Text("Ciao"), new DoubleWritable(1.123d)),
+        EdgeFactory.create(new Text("Bomdia"), new DoubleWritable(2.234d)),
+        EdgeFactory.create(new Text("Ola"), new DoubleWritable(3.345d)));
     assertEquals(vertex.getNumEdges(), 3);
   }
 
@@ -201,9 +201,9 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex,
         new Text("BYE"), new DoubleWritable(0.01d),
-        new DefaultEdge<Text, DoubleWritable>(new Text("CIAO"), new DoubleWritable(1.001d)),
-        new DefaultEdge<Text, DoubleWritable>(new Text("TCHAU"), new DoubleWritable(2.0001d)),
-        new DefaultEdge<Text, DoubleWritable>(new Text("ADIOS"), new DoubleWritable(3.00001d)));
+        EdgeFactory.create(new Text("CIAO"), new DoubleWritable(1.001d)),
+        EdgeFactory.create(new Text("TCHAU"), new DoubleWritable(2.0001d)),
+        EdgeFactory.create(new Text("ADIOS"), new DoubleWritable(3.00001d)));
 
     assertEquals(vertex.getNumEdges(), 3);
   }
@@ -222,7 +222,7 @@ public class TestTextDoubleDoubleAdjacencyListVertexInputFormat extends TextDoub
         vr.getCurrentVertex();
     setGraphState(vertex, graphState);
     assertValidVertex(conf, graphState, vertex, new Text("alpha"), new DoubleWritable(42d),
-        new DefaultEdge<Text, DoubleWritable>(new Text("beta"), new DoubleWritable(99d)));
+        EdgeFactory.create(new Text("beta"), new DoubleWritable(99d)));
     assertEquals(vertex.getNumEdges(), 1);
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java b/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
index a0b16e2..2671708 100644
--- a/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
+++ b/giraph-core/src/test/java/org/apache/giraph/partition/TestGiraphTransferRegulator.java
@@ -17,10 +17,10 @@
  */
 package org.apache.giraph.partition;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.job.GiraphJob;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.graph.GiraphTransferRegulator;
+import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -75,12 +75,9 @@ public class TestGiraphTransferRegulator {
     job.getConfiguration()
         .setInt(GiraphTransferRegulator.MAX_EDGES_PER_TRANSFER, 3);
     List<Edge<IntWritable, DoubleWritable>> edges = Lists.newLinkedList();
-    edges.add(new DefaultEdge<IntWritable, DoubleWritable>(new IntWritable(2),
-        new DoubleWritable(22)));
-    edges.add(new DefaultEdge<IntWritable, DoubleWritable>(new IntWritable(3),
-        new DoubleWritable(33)));
-    edges.add(new DefaultEdge<IntWritable, DoubleWritable>(new IntWritable(4),
-        new DoubleWritable(44)));
+    edges.add(EdgeFactory.create(new IntWritable(2), new DoubleWritable(22)));
+    edges.add(EdgeFactory.create(new IntWritable(3), new DoubleWritable(33)));
+    edges.add(EdgeFactory.create(new IntWritable(4), new DoubleWritable(44)));
     vertex.initialize(null, null, edges);
     GiraphTransferRegulator gtr =
         new GiraphTransferRegulator(job.getConfiguration());

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/test/java/org/apache/giraph/vertex/TestIntIntNullIntVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/vertex/TestIntIntNullIntVertex.java b/giraph-core/src/test/java/org/apache/giraph/vertex/TestIntIntNullIntVertex.java
index 155861c..37e8768 100644
--- a/giraph-core/src/test/java/org/apache/giraph/vertex/TestIntIntNullIntVertex.java
+++ b/giraph-core/src/test/java/org/apache/giraph/vertex/TestIntIntNullIntVertex.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.vertex;
 
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.junit.Test;
@@ -55,8 +55,8 @@ public class TestIntIntNullIntVertex {
     IntIntNullIntVertex vertex = new MyIntIntNullVertex();
 
     List<Edge<IntWritable, NullWritable>> edges = Lists.newLinkedList();
-    edges.add(new EdgeNoValue<IntWritable>(new IntWritable(3)));
-    edges.add(new EdgeNoValue<IntWritable>(new IntWritable(47)));
+    edges.add(EdgeFactory.create(new IntWritable(3)));
+    edges.add(EdgeFactory.create(new IntWritable(47)));
 
     vertex.initialize(new IntWritable(23), new IntWritable(7), edges);
     vertex.voteToHalt();

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/test/java/org/apache/giraph/vertex/TestMultiGraphVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/vertex/TestMultiGraphVertex.java b/giraph-core/src/test/java/org/apache/giraph/vertex/TestMultiGraphVertex.java
index 11fc8a5..173c453 100644
--- a/giraph-core/src/test/java/org/apache/giraph/vertex/TestMultiGraphVertex.java
+++ b/giraph-core/src/test/java/org/apache/giraph/vertex/TestMultiGraphVertex.java
@@ -18,14 +18,15 @@
 
 package org.apache.giraph.vertex;
 
-import com.google.common.collect.Lists;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.util.Collection;
 
@@ -77,36 +78,27 @@ public class TestMultiGraphVertex {
     // in order to catch corner cases:
 
     // Edge list of form: [A, B, A]
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(1),
-        new IntWritable(1)));
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(2),
-        new IntWritable(2)));
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(1),
-        new IntWritable(10)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(1), new IntWritable(1)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(2), new IntWritable(2)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(1), new IntWritable(10)));
     assertEquals(vertex.getNumEdges(), 3);
     assertEquals(vertex.removeEdges(new IntWritable(1)), 2);
     assertEquals(vertex.getNumEdges(), 1);
 
     // Edge list of form: [A, B, B]
     vertex = instantiateVertex(vertexClass);
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(2),
-        new IntWritable(2)));
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(1),
-        new IntWritable(1)));
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(1),
-        new IntWritable(10)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(2), new IntWritable(2)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(1), new IntWritable(1)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(1), new IntWritable(10)));
     assertEquals(vertex.getNumEdges(), 3);
     assertEquals(vertex.removeEdges(new IntWritable(1)), 2);
     assertEquals(vertex.getNumEdges(), 1);
 
     // Edge list of form: [A, A, B]
     vertex = instantiateVertex(vertexClass);
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(1),
-        new IntWritable(1)));
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(1),
-        new IntWritable(10)));
-    vertex.addEdge(new DefaultEdge<IntWritable, IntWritable>(new IntWritable(2),
-        new IntWritable(2)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(1), new IntWritable(1)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(1), new IntWritable(10)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(2), new IntWritable(2)));
     assertEquals(vertex.getNumEdges(), 3);
     assertEquals(vertex.removeEdges(new IntWritable(1)), 2);
     assertEquals(vertex.getNumEdges(), 1);

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-core/src/test/java/org/apache/giraph/vertex/TestMutableVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/vertex/TestMutableVertex.java b/giraph-core/src/test/java/org/apache/giraph/vertex/TestMutableVertex.java
index cbbaa1b..2b347c6 100644
--- a/giraph-core/src/test/java/org/apache/giraph/vertex/TestMutableVertex.java
+++ b/giraph-core/src/test/java/org/apache/giraph/vertex/TestMutableVertex.java
@@ -19,8 +19,8 @@ package org.apache.giraph.vertex;
 
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.time.SystemTime;
 import org.apache.giraph.time.Time;
 import org.apache.giraph.time.Times;
@@ -147,8 +147,8 @@ public class TestMutableVertex {
 
     List<Edge<IntWritable, DoubleWritable>> edges = Lists.newLinkedList();
     for (int i = 1000; i > 0; --i) {
-      edges.add(new DefaultEdge<IntWritable, DoubleWritable>(
-          new IntWritable(i), new DoubleWritable(i * 2.0)));
+      edges.add(EdgeFactory.create(new IntWritable(i),
+          new DoubleWritable(i * 2.0)));
     }
 
     vertex.initialize(null, null, edges);
@@ -182,7 +182,7 @@ public class TestMutableVertex {
 
     List<Edge<IntWritable, DoubleWritable>> edges = Lists.newLinkedList();
     for (int i = 1000; i > 0; --i) {
-      edges.add(new DefaultEdge<IntWritable, DoubleWritable>(
+      edges.add(EdgeFactory.create(
           new IntWritable(i), new DoubleWritable(i * 3.0)));
     }
 
@@ -220,19 +220,19 @@ public class TestMutableVertex {
 
     vertex.initialize(new IntWritable(0), new FloatWritable(0.0f));
     assertEquals(vertex.getNumEdges(), 0);
-    assertTrue(vertex.addEdge(new DefaultEdge<IntWritable, DoubleWritable>(
+    assertTrue(vertex.addEdge(EdgeFactory.create(
         new IntWritable(2),
         new DoubleWritable(2.0))));
     assertEquals(vertex.getNumEdges(), 1);
     assertEquals(vertex.getEdgeValue(new IntWritable(2)),
         new DoubleWritable(2.0));
-    assertTrue(vertex.addEdge(new DefaultEdge<IntWritable, DoubleWritable>(
+    assertTrue(vertex.addEdge(EdgeFactory.create(
         new IntWritable(4),
         new DoubleWritable(4.0))));
-    assertTrue(vertex.addEdge(new DefaultEdge<IntWritable, DoubleWritable>(
+    assertTrue(vertex.addEdge(EdgeFactory.create(
         new IntWritable(3),
         new DoubleWritable(3.0))));
-    assertTrue(vertex.addEdge(new DefaultEdge<IntWritable, DoubleWritable>(
+    assertTrue(vertex.addEdge(EdgeFactory.create(
         new IntWritable(1),
         new DoubleWritable(1.0))));
     assertEquals(vertex.getNumEdges(), 4);
@@ -280,7 +280,7 @@ public class TestMutableVertex {
     List<Edge<IntWritable, DoubleWritable>> edges =
         Lists.newArrayListWithCapacity(edgesCount);
     for (int i = edgesCount; i > 0; --i) {
-      edges.add(new DefaultEdge<IntWritable, DoubleWritable>(
+      edges.add(EdgeFactory.create(
           new IntWritable(i), new DoubleWritable(i * 2.0)));
     }
     vertex.initialize(new IntWritable(2), new FloatWritable(3.0f), edges);

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
index 100687b..b1305cc 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleFloatDoubleTextInputFormat.java
@@ -19,8 +19,8 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.bsp.BspUtils;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.giraph.vertex.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
@@ -74,7 +74,7 @@ public class LongDoubleFloatDoubleTextInputFormat
           Lists.newArrayListWithCapacity(tokens.length - 1);
       float weight = 1.0f / (tokens.length - 1);
       for (int n = 1; n < tokens.length; n++) {
-        edges.add(new DefaultEdge<LongWritable, FloatWritable>(
+        edges.add(EdgeFactory.create(
             new LongWritable(Long.parseLong(tokens[n])),
             new FloatWritable(weight)));
       }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
index 592d78a..1d41a94 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleFloatDoubleTextInputFormat.java
@@ -19,8 +19,8 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.bsp.BspUtils;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.giraph.vertex.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
@@ -94,8 +94,7 @@ public class NormalizingLongDoubleFloatDoubleTextInputFormat
                Collection<Edge<LongWritable, FloatWritable>> edges) {
       for (int n = 1; n < tokens.length; n++) {
         String[] parts = weightSeparator.split(tokens[n]);
-        edges.add(new DefaultEdge<LongWritable, FloatWritable>(
-            new LongWritable(Long.parseLong(parts[0])),
+        edges.add(EdgeFactory.create(new LongWritable(Long.parseLong(parts[0])),
             new FloatWritable(Float.parseFloat(parts[1]))));
       }
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
index 337f30e..09c7f56 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
@@ -24,11 +24,11 @@ import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.job.GiraphJob;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
+import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.giraph.worker.WorkerContext;
@@ -114,8 +114,7 @@ public class SimpleCheckpointVertex implements Tool {
         FloatWritable newEdgeValue = new FloatWritable(edge.getValue().get() +
             (float) vertexValue);
         Edge<LongWritable, FloatWritable> newEdge =
-            new DefaultEdge<LongWritable, FloatWritable>(
-                edge.getTargetVertexId(), newEdgeValue);
+            EdgeFactory.create(edge.getTargetVertexId(), newEdgeValue);
         LOG.info("compute: vertex " + getId() +
             " sending edgeValue " + edge.getValue() +
             " vertexValue " + vertexValue +

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
index d3bd33d..6032e88 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleMutateGraphVertex.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.DefaultEdge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.io.DoubleWritable;
@@ -87,8 +87,7 @@ public class SimpleMutateGraphVertex extends EdgeListVertex<
       addVertexRequest(vertexIndex, new DoubleWritable(0.0));
       // Add edges to those remote vertices as well
       addEdgeRequest(vertexIndex,
-          new DefaultEdge<LongWritable, FloatWritable>(
-              getId(), new FloatWritable(0.0f)));
+          EdgeFactory.create(getId(), new FloatWritable(0.0f)));
     } else if (getSuperstep() == 4) {
       LOG.debug("Reached superstep " + getSuperstep());
     } else if (getSuperstep() == 5) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 ba0242d..f36f1ac 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
@@ -21,8 +21,8 @@ package org.apache.giraph.examples;
 import org.apache.giraph.aggregators.DoubleMaxAggregator;
 import org.apache.giraph.aggregators.DoubleMinAggregator;
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
 import org.apache.giraph.io.formats.TextVertexOutputFormat;
@@ -196,8 +196,7 @@ public class SimplePageRankVertex extends LongDoubleFloatDoubleVertex {
           (inputSplit.getNumSplits() * totalRecords);
       float edgeValue = vertexId.get() * 100f;
       List<Edge<LongWritable, FloatWritable>> edges = Lists.newLinkedList();
-      edges.add(new DefaultEdge<LongWritable, FloatWritable>(
-          new LongWritable(targetVertexId),
+      edges.add(EdgeFactory.create(new LongWritable(targetVertexId),
           new FloatWritable(edgeValue)));
       vertex.initialize(vertexId, vertexValue, edges);
       ++recordsRead;

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 09efe88..2afe15d 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
@@ -18,8 +18,8 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
 import org.apache.giraph.io.formats.TextVertexOutputFormat;
@@ -98,8 +98,7 @@ public class SimpleSuperstepVertex extends
           (vertexId.get() + 1) %
           (inputSplit.getNumSplits() * totalRecords);
       float edgeValue = vertexId.get() * 100f;
-      edges.add(new DefaultEdge<LongWritable, FloatWritable>(
-          new LongWritable(targetVertexId),
+      edges.add(EdgeFactory.create(new LongWritable(targetVertexId),
           new FloatWritable(edgeValue)));
       vertex.initialize(vertexId, vertexValue, edges);
       ++recordsRead;

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java b/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
index 507a56b..994b59d 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/VerifyMessage.java
@@ -19,8 +19,8 @@
 package org.apache.giraph.examples;
 
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.master.DefaultMasterCompute;
 import org.apache.giraph.vertex.EdgeListVertex;
 import org.apache.giraph.worker.WorkerContext;
@@ -187,9 +187,7 @@ public class VerifyMessage {
         FloatWritable newEdgeValue = new FloatWritable(
             edge.getValue().get() + (float) vertexValue);
         Edge<LongWritable, FloatWritable> newEdge =
-            new DefaultEdge<LongWritable, FloatWritable>(
-                edge.getTargetVertexId(),
-                newEdgeValue);
+            EdgeFactory.create(edge.getTargetVertexId(), newEdgeValue);
         if (LOG.isDebugEnabled()) {
           LOG.debug("compute: vertex " + getId() +
               " sending edgeValue " + edge.getValue() +

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
index 7a7b148..a372a2d 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsVertexTest.java
@@ -18,11 +18,8 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.giraph.conf.GiraphClasses;
-import org.apache.giraph.graph.DefaultEdge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexInputFormat;
 import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexOutputFormat;
 import org.apache.giraph.utils.InternalVertexRunner;
@@ -35,9 +32,15 @@ import org.json.JSONException;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 import java.util.Map;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Contains a simple unit test for {@link SimpleShortestPathsVertex}
@@ -52,9 +55,9 @@ public class SimpleShortestPathsVertexTest {
 
     SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex();
     vertex.initialize(null, null);
-    vertex.addEdge(new DefaultEdge<LongWritable, FloatWritable>(
+    vertex.addEdge(EdgeFactory.create(
         new LongWritable(10L), new FloatWritable(2.5f)));
-    vertex.addEdge(new DefaultEdge<LongWritable, FloatWritable>(
+    vertex.addEdge(EdgeFactory.create(
         new LongWritable(20L), new FloatWritable(0.5f)));
 
     MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,
@@ -85,9 +88,9 @@ public class SimpleShortestPathsVertexTest {
 
     SimpleShortestPathsVertex vertex = new SimpleShortestPathsVertex();
     vertex.initialize(new LongWritable(0), new DoubleWritable(0.0));
-    vertex.addEdge(new DefaultEdge<LongWritable, FloatWritable>(
-        new LongWritable(10L), new FloatWritable(2.5f)));
-    vertex.addEdge(new DefaultEdge<LongWritable, FloatWritable>(
+    vertex.addEdge(EdgeFactory.create(new LongWritable(10L),
+        new FloatWritable(2.5f)));
+    vertex.addEdge(EdgeFactory.create(
         new LongWritable(20L), new FloatWritable(0.5f)));
 
     MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
index 7e7b13d..969c1f4 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingVertexTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.examples;
 
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.utils.MockUtils;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -45,8 +45,8 @@ public class SimpleTriangleClosingVertexTest {
     SimpleTriangleClosingVertex.IntArrayListWritable alw =
       new SimpleTriangleClosingVertex.IntArrayListWritable();
     vertex.initialize(null, null);
-    vertex.addEdge(new EdgeNoValue<IntWritable>(new IntWritable(5)));
-    vertex.addEdge(new EdgeNoValue<IntWritable>(new IntWritable(7)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(5)));
+    vertex.addEdge(EdgeFactory.create(new IntWritable(7)));
 
     MockUtils.MockedEnvironment<IntWritable,
       SimpleTriangleClosingVertex.IntArrayListWritable,

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 a823971..f45f07a 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
@@ -17,8 +17,8 @@
  */
 package org.apache.giraph.io.hbase.edgemarker;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.io.hbase.HBaseVertexInputFormat;
 import org.apache.giraph.vertex.Vertex;
@@ -87,7 +87,7 @@ public class TableEdgeInputFormat extends
             String edge = Bytes.toString(row.getValue(CF, CHILDREN));
             Text vertexValue = new Text();
             Text edgeId = new Text(edge);
-            edges.add(new DefaultEdge<Text, Text>(edgeId, uselessEdgeValue));
+            edges.add(EdgeFactory.create(edgeId, uselessEdgeValue));
             vertex.initialize(vertexId, vertexValue, edges);
 
             return vertex;

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogEdgeInputFormat.java b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogEdgeInputFormat.java
index fe0ddd5..8d85056 100644
--- a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogEdgeInputFormat.java
+++ b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogEdgeInputFormat.java
@@ -18,9 +18,8 @@
 
 package org.apache.giraph.io.hcatalog;
 
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.EdgeNoValue;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.EdgeReader;
 import org.apache.hadoop.io.NullWritable;
@@ -202,7 +201,7 @@ public abstract class HCatalogEdgeInputFormat<
     public Edge<I, E> getCurrentEdge() throws IOException,
         InterruptedException {
       HCatRecord record = getRecordReader().getCurrentValue();
-      return new DefaultEdge<I, E>(getTargetVertexId(record),
+      return EdgeFactory.create(getTargetVertexId(record),
           getEdgeValue(record));
     }
   }
@@ -249,7 +248,7 @@ public abstract class HCatalogEdgeInputFormat<
     public Edge<I, NullWritable> getCurrentEdge() throws IOException,
         InterruptedException {
       HCatRecord record = getRecordReader().getCurrentValue();
-      return new EdgeNoValue<I>(getTargetVertexId(record));
+      return EdgeFactory.create(getTargetVertexId(record));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/028b0c92/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 319242d..e1d9791 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
@@ -19,8 +19,8 @@
 package org.apache.giraph.io.hcatalog;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.graph.DefaultEdge;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.EdgeFactory;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.utils.TimedLogger;
@@ -349,15 +349,13 @@ public abstract class HCatalogVertexInputFormat<
     @Override
     public boolean nextVertex() throws IOException, InterruptedException {
       while (getRecordReader().nextKeyValue()) {
-        HCatRecord record =
-            getRecordReader().getCurrentValue();
+        HCatRecord record = getRecordReader().getCurrentValue();
         if (currentVertexId == null) {
           currentVertexId = getVertexId(record);
         }
         if (currentVertexId.equals(getVertexId(record))) {
-          currentEdges.add(new DefaultEdge<I, E>(
-                  getTargetVertexId(record),
-                  getEdgeValue(record)));
+          currentEdges.add(EdgeFactory.create(getTargetVertexId(record),
+              getEdgeValue(record)));
           recordsForVertex.add(record);
         } else {
           createCurrentVertex();