You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ap...@apache.org on 2013/04/16 03:12:05 UTC

[1/2] GIRAPH-633: Rename VertexEdges to OutEdges (apresta)

Updated Branches:
  refs/heads/trunk 38f1f1793 -> 158c0b118


http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 6703b22..5d293eb 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
@@ -25,7 +25,7 @@ import java.util.Random;
 import java.util.Set;
 import org.apache.giraph.bsp.BspInputSplit;
 import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexReader;
@@ -144,9 +144,9 @@ public class PseudoRandomVertexInputFormat extends
       Random rand = new Random(vertexId);
       DoubleWritable vertexValue = new DoubleWritable(rand.nextDouble());
       // In order to save memory and avoid copying, we add directly to a
-      // VertexEdges instance.
-      VertexEdges<LongWritable, DoubleWritable> edges =
-          getConf().createAndInitializeVertexEdges(edgesPerVertex);
+      // OutEdges instance.
+      OutEdges<LongWritable, DoubleWritable> edges =
+          getConf().createAndInitializeOutEdges(edgesPerVertex);
       Set<LongWritable> destVertices = Sets.newHashSet();
       for (long i = 0; i < edgesPerVertex; ++i) {
         LongWritable destVertexId = new LongWritable();

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 54b6bbd..cc6b126 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
@@ -21,7 +21,7 @@ package org.apache.giraph.job;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
 import org.apache.giraph.graph.DefaultVertexValueFactory;
 import org.apache.giraph.graph.Vertex;
@@ -118,7 +118,7 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
     vertexValueType = classList.get(VALUE_PARAM_INDEX);
     edgeValueType = classList.get(EDGE_PARAM_INDEX);
     messageValueType = classList.get(MSG_PARAM_INDEX);
-    verifyVertexEdgesGenericTypes();
+    verifyOutEdgesGenericTypes();
     verifyVertexInputFormatGenericTypes();
     verifyEdgeInputFormatGenericTypes();
     verifyVertexOutputFormatGenericTypes();
@@ -165,7 +165,7 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
             VERTEX_RESOLVER_CLASS.getDefaultClass().getCanonicalName());
       }
     }
-    if (conf.getVertexEdgesClass() == null) {
+    if (conf.getOutEdgesClass() == null) {
       if (LOG.isInfoEnabled()) {
         LOG.info("checkConfiguration: No class found for " +
             VERTEX_EDGES_CLASS.getKey() + ", defaulting to " +
@@ -175,15 +175,15 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
   }
 
   /**
-   * Verify matching generic types for a specific VertexEdges class.
+   * Verify matching generic types for a specific OutEdges class.
    *
-   * @param vertexEdgesClass {@link VertexEdges} class to check
+   * @param outEdgesClass {@link org.apache.giraph.edge.OutEdges} class to check
    */
-  private void verifyVertexEdgesGenericTypesClass(
-      Class<? extends VertexEdges<I, E>> vertexEdgesClass) {
+  private void verifyOutEdgesGenericTypesClass(
+      Class<? extends OutEdges<I, E>> outEdgesClass) {
     List<Class<?>> classList = ReflectionUtils.getTypeArguments(
-        VertexEdges.class, vertexEdgesClass);
-    // VertexEdges implementations can be generic, in which case there are no
+        OutEdges.class, outEdgesClass);
+    // OutEdges implementations can be generic, in which case there are no
     // types to check.
     if (classList.isEmpty()) {
       return;
@@ -205,14 +205,14 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
     }
   }
 
-  /** Verify matching generic types in VertexEdges. */
-  private void verifyVertexEdgesGenericTypes() {
-    Class<? extends VertexEdges<I, E>> vertexEdgesClass =
-        conf.getVertexEdgesClass();
-    Class<? extends VertexEdges<I, E>> inputVertexEdgesClass =
-        conf.getInputVertexEdgesClass();
-    verifyVertexEdgesGenericTypesClass(vertexEdgesClass);
-    verifyVertexEdgesGenericTypesClass(inputVertexEdgesClass);
+  /** Verify matching generic types in OutEdges. */
+  private void verifyOutEdgesGenericTypes() {
+    Class<? extends OutEdges<I, E>> outEdgesClass =
+        conf.getOutEdgesClass();
+    Class<? extends OutEdges<I, E>> inputOutEdgesClass =
+        conf.getInputOutEdgesClass();
+    verifyOutEdgesGenericTypesClass(outEdgesClass);
+    verifyOutEdgesGenericTypesClass(inputOutEdgesClass);
   }
 
   /** Verify matching generic types in VertexInputFormat. */

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java b/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
index 7c7fc2b..c4669d3 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/ByteArrayPartition.java
@@ -73,7 +73,7 @@ public class ByteArrayPartition<I extends WritableComparable,
     representativeVertex.initialize(
         getConf().createVertexId(),
         getConf().createVertexValue(),
-        getConf().createVertexEdges());
+        getConf().createOutEdges());
     useUnsafeSerialization = getConf().useUnsafeSerialization();
   }
 
@@ -194,7 +194,7 @@ public class ByteArrayPartition<I extends WritableComparable,
     representativeVertex.initialize(
         getConf().createVertexId(),
         getConf().createVertexValue(),
-        getConf().createVertexEdges());
+        getConf().createOutEdges());
     useUnsafeSerialization = getConf().useUnsafeSerialization();
     for (int i = 0; i < size; ++i) {
       progress();

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java b/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
index 53b9dd4..2d30bf9 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/DiskBackedPartitionStore.java
@@ -19,7 +19,7 @@
 package org.apache.giraph.partition;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -381,12 +381,12 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
-  private void writeVertexEdges(
+  private void writeOutEdges(
       DataOutput output,
       Vertex<I, V, E, M> vertex)
     throws IOException {
     vertex.getId().write(output);
-    ((VertexEdges<I, E>) vertex.getEdges()).write(output);
+    ((OutEdges<I, E>) vertex.getEdges()).write(output);
   }
 
   /**
@@ -417,12 +417,12 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @param partition The partition owning the vertex
    * @throws IOException
    */
-  private void readVertexEdges(DataInput in, Partition<I, V, E, M> partition)
+  private void readOutEdges(DataInput in, Partition<I, V, E, M> partition)
     throws IOException {
     I id = conf.createVertexId();
     id.readFields(in);
     Vertex<I, V, E, M> v = partition.getVertex(id);
-    VertexEdges<I, E> edges = conf.createVertexEdges();
+    OutEdges<I, E> edges = conf.createOutEdges();
     edges.readFields(in);
     v.setEdges(edges);
   }
@@ -454,7 +454,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
     inputStream = new DataInputStream(
         new BufferedInputStream(new FileInputStream(file)));
     for (int i = 0; i < numVertices; ++i) {
-      readVertexEdges(inputStream, partition);
+      readOutEdges(inputStream, partition);
     }
     inputStream.close();
     /*
@@ -501,7 +501,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
       outputStream = new DataOutputStream(
           new BufferedOutputStream(new FileOutputStream(file)));
       for (Vertex<I, V, E, M> vertex : partition) {
-        writeVertexEdges(outputStream, vertex);
+        writeOutEdges(outputStream, vertex);
       }
       outputStream.close();
     }
@@ -530,7 +530,7 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
     outputStream = new DataOutputStream(
         new BufferedOutputStream(new FileOutputStream(file, true)));
     for (Vertex<I, V, E, M> vertex : partition) {
-      writeVertexEdges(outputStream, vertex);
+      writeOutEdges(outputStream, vertex);
     }
     outputStream.close();
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
index c9a52d1..6016ba4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
@@ -30,6 +30,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
@@ -39,7 +40,6 @@ import org.apache.giraph.job.GiraphConfigurationValidator;
 import org.apache.giraph.master.MasterCompute;
 import org.apache.giraph.partition.Partition;
 import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.edge.VertexEdges;
 import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -91,7 +91,7 @@ public final class ConfigurationUtils {
     OPTIONS.addOption("eip", "edgeInputPath", true, "Edge input path");
     OPTIONS.addOption("op", "outputPath", true, "Vertex output path");
     OPTIONS.addOption("c", "combiner", true, "Combiner class");
-    OPTIONS.addOption("ve", "vertexEdges", true, "Vertex edges class");
+    OPTIONS.addOption("ve", "outEdges", true, "Vertex edges class");
     OPTIONS.addOption("wc", "workerContext", true, "WorkerContext class");
     OPTIONS.addOption("aw", "aggregatorWriter", true, "AggregatorWriter class");
     OPTIONS.addOption("mc", "masterCompute", true, "MasterCompute class");
@@ -214,13 +214,13 @@ public final class ConfigurationUtils {
           (Class<? extends Combiner>) Class.forName(cmd.getOptionValue("c")));
     }
     if (cmd.hasOption("ve")) {
-      giraphConfiguration.setVertexEdgesClass(
-          (Class<? extends VertexEdges>)
+      giraphConfiguration.setOutEdgesClass(
+          (Class<? extends OutEdges>)
               Class.forName(cmd.getOptionValue("ve")));
     }
     if (cmd.hasOption("ive")) {
-      giraphConfiguration.setInputVertexEdgesClass(
-          (Class<? extends VertexEdges>)
+      giraphConfiguration.setInputOutEdgesClass(
+          (Class<? extends OutEdges>)
               Class.forName(cmd.getOptionValue("ive")));
     }
     if (cmd.hasOption("wc")) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 5391aa0..c607ca3 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
@@ -20,7 +20,7 @@ package org.apache.giraph.utils;
 
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.zk.ZooKeeperExt;
 import org.apache.giraph.zk.ZooKeeperExt.PathStat;
@@ -477,7 +477,7 @@ public class WritableUtils {
     throws IOException {
     vertex.getId().readFields(input);
     vertex.getValue().readFields(input);
-    ((VertexEdges<I, E>) vertex.getEdges()).readFields(input);
+    ((OutEdges<I, E>) vertex.getEdges()).readFields(input);
     if (input.readBoolean()) {
       vertex.voteToHalt();
     } else {
@@ -506,7 +506,7 @@ public class WritableUtils {
     Vertex<I, V, E, M> vertex = conf.createVertex();
     I id = conf.createVertexId();
     V value = conf.createVertexValue();
-    VertexEdges<I, E> edges = conf.createVertexEdges();
+    OutEdges<I, E> edges = conf.createOutEdges();
     vertex.initialize(id, value, edges);
     reinitializeVertexFromDataInput(input, vertex, conf);
     return vertex;
@@ -533,7 +533,7 @@ public class WritableUtils {
     throws IOException {
     vertex.getId().write(output);
     vertex.getValue().write(output);
-    ((VertexEdges<I, E>) vertex.getEdges()).write(output);
+    ((OutEdges<I, E>) vertex.getEdges()).write(output);
     output.writeBoolean(vertex.isHalted());
   }
 }

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
index 4eeff01..2bb3292 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiGraphEdges.java
@@ -27,15 +27,15 @@ import org.junit.Test;
 import java.util.Collection;
 import java.util.List;
 
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
 import static org.junit.Assert.assertEquals;
 
 /**
- * Tests {@link VertexEdges} implementations that allow parallel edges.
+ * Tests {@link OutEdges} implementations that allow parallel edges.
  */
 public class TestMultiGraphEdges {
-  /** {@link VertexEdges} classes to be tested. */
-  private Collection<Class<? extends VertexEdges>> edgesClasses =
+  /** {@link OutEdges} classes to be tested. */
+  private Collection<Class<? extends OutEdges>> edgesClasses =
       Lists.newArrayList();
 
   @Before
@@ -47,20 +47,20 @@ public class TestMultiGraphEdges {
   }
 
   /**
-   * Ensures that all multigraph {@link VertexEdges} implementations allow
+   * Ensures that all multigraph {@link OutEdges} implementations allow
    * parallel edges.
    */
   @Test
   public void testParallelEdges() {
-    for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+    for (Class<? extends OutEdges> edgesClass : edgesClasses) {
       testParallelEdgesClass(edgesClass);
     }
   }
 
   private void testParallelEdgesClass(
-      Class<? extends VertexEdges> edgesClass) {
-    VertexEdges<LongWritable, DoubleWritable> edges =
-        instantiateVertexEdges(edgesClass);
+      Class<? extends OutEdges> edgesClass) {
+    OutEdges<LongWritable, DoubleWritable> edges =
+        instantiateOutEdges(edgesClass);
 
     // Initial edges list contains parallel edges.
     List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
index b86f409..c300d11 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestMultiRandomAccessEdges.java
@@ -28,15 +28,15 @@ import org.junit.Test;
 import java.util.Collection;
 import java.util.List;
 
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
 import static org.junit.Assert.assertEquals;
 
 /**
- * Tests {@link MultiRandomAccessVertexEdges} implementations.
+ * Tests {@link MultiRandomAccessOutEdges} implementations.
  */
 public class TestMultiRandomAccessEdges {
-  /** {@link MultiRandomAccessVertexEdges} classes to be tested. */
-  private Collection<Class<? extends MultiRandomAccessVertexEdges>>
+  /** {@link MultiRandomAccessOutEdges} classes to be tested. */
+  private Collection<Class<? extends MultiRandomAccessOutEdges>>
       edgesClasses = Lists.newArrayList();
 
   @Before
@@ -45,22 +45,22 @@ public class TestMultiRandomAccessEdges {
   }
 
   /**
-   * Ensures that all {@link MultiRandomAccessVertexEdges} implementations
+   * Ensures that all {@link MultiRandomAccessOutEdges} implementations
    * correctly return edge values.
    */
   @Test
   public void testParallelEdges() {
-    for (Class<? extends MultiRandomAccessVertexEdges> edgesClass :
+    for (Class<? extends MultiRandomAccessOutEdges> edgesClass :
         edgesClasses) {
       testParallelEdgesClass(edgesClass);
     }
   }
 
   private void testParallelEdgesClass(
-      Class<? extends MultiRandomAccessVertexEdges> edgesClass) {
-    MultiRandomAccessVertexEdges<LongWritable, DoubleWritable> edges =
-        (MultiRandomAccessVertexEdges<LongWritable, DoubleWritable>)
-            instantiateVertexEdges(edgesClass);
+      Class<? extends MultiRandomAccessOutEdges> edgesClass) {
+    MultiRandomAccessOutEdges<LongWritable, DoubleWritable> edges =
+        (MultiRandomAccessOutEdges<LongWritable, DoubleWritable>)
+            instantiateOutEdges(edgesClass);
 
     // Initial edges list contains parallel edges.
     List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
index 79bf093..99a104d 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestNullValueEdges.java
@@ -28,15 +28,15 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
 import static org.junit.Assert.assertEquals;
 
 /**
- * Tests {@link VertexEdges} implementations with null edge values.
+ * Tests {@link OutEdges} implementations with null edge values.
  */
 public class TestNullValueEdges {
-  /** {@link VertexEdges} classes to be tested. */
-  private Collection<Class<? extends MutableVertexEdges>>
+  /** {@link OutEdges} classes to be tested. */
+  private Collection<Class<? extends MutableOutEdges>>
       edgesClasses = Lists.newArrayList();
 
   @Before
@@ -47,16 +47,16 @@ public class TestNullValueEdges {
 
   @Test
   public void testEdges() {
-    for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+    for (Class<? extends OutEdges> edgesClass : edgesClasses) {
       testEdgesClass(edgesClass);
     }
   }
 
   private void testEdgesClass(
-      Class<? extends VertexEdges> edgesClass) {
-    VertexEdges<LongWritable, NullWritable> edges =
-        (VertexEdges<LongWritable, NullWritable>)
-            instantiateVertexEdges(edgesClass);
+      Class<? extends OutEdges> edgesClass) {
+    OutEdges<LongWritable, NullWritable> edges =
+        (OutEdges<LongWritable, NullWritable>)
+            instantiateOutEdges(edgesClass);
 
     List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
         EdgeFactory.create(new LongWritable(1)),
@@ -79,16 +79,16 @@ public class TestNullValueEdges {
    */
   @Test
   public void testMutateEdges() {
-    for (Class<? extends MutableVertexEdges> edgesClass : edgesClasses) {
+    for (Class<? extends MutableOutEdges> edgesClass : edgesClasses) {
       testMutateEdgesClass(edgesClass);
     }
   }
 
   private void testMutateEdgesClass(
-      Class<? extends MutableVertexEdges> edgesClass) {
-    MutableVertexEdges<LongWritable, NullWritable> edges =
-       (MutableVertexEdges<LongWritable, NullWritable>)
-           instantiateVertexEdges(edgesClass);
+      Class<? extends MutableOutEdges> edgesClass) {
+    MutableOutEdges<LongWritable, NullWritable> edges =
+       (MutableOutEdges<LongWritable, NullWritable>)
+           instantiateOutEdges(edgesClass);
 
     edges.initialize();
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
index 3c57281..a06de0a 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictGraphEdges.java
@@ -27,15 +27,15 @@ import org.junit.Test;
 import java.util.Collection;
 import java.util.List;
 
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
 import static org.junit.Assert.assertEquals;
 
 /**
- * Tests {@link VertexEdges} implementations that disallow parallel edges.
+ * Tests {@link OutEdges} implementations that disallow parallel edges.
  */
 public class TestStrictGraphEdges {
-  /** {@link VertexEdges} classes to be tested. */
-  private Collection<Class<? extends VertexEdges>> edgesClasses =
+  /** {@link OutEdges} classes to be tested. */
+  private Collection<Class<? extends OutEdges>> edgesClasses =
       Lists.newArrayList();
 
   @Before
@@ -45,20 +45,20 @@ public class TestStrictGraphEdges {
   }
 
   /**
-   * Ensures that all strict graph {@link VertexEdges} implementations
+   * Ensures that all strict graph {@link OutEdges} implementations
    * disallow parallel edges.
    */
   @Test
   public void testParallelEdges() {
-    for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+    for (Class<? extends OutEdges> edgesClass : edgesClasses) {
       testParallelEdgesClass(edgesClass);
     }
   }
 
   private void testParallelEdgesClass(
-      Class<? extends VertexEdges> edgesClass) {
-    VertexEdges<LongWritable, DoubleWritable> edges =
-        instantiateVertexEdges(edgesClass);
+      Class<? extends OutEdges> edgesClass) {
+    OutEdges<LongWritable, DoubleWritable> edges =
+        instantiateOutEdges(edgesClass);
 
     // Initial edges list contains parallel edges.
     List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
index d3ad069..74d7054 100644
--- a/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/TestStrictRandomAccessEdges.java
@@ -27,16 +27,16 @@ import org.junit.Test;
 import java.util.Collection;
 import java.util.List;
 
-import static org.apache.giraph.graph.TestVertexAndEdges.instantiateVertexEdges;
+import static org.apache.giraph.graph.TestVertexAndEdges.instantiateOutEdges;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
 /**
- * Tests {@link StrictRandomAccessVertexEdges} implementations.
+ * Tests {@link StrictRandomAccessOutEdges} implementations.
  */
 public class TestStrictRandomAccessEdges {
-    /** {@link StrictRandomAccessVertexEdges} classes to be tested. */
-    private Collection<Class<? extends StrictRandomAccessVertexEdges>>
+    /** {@link StrictRandomAccessOutEdges} classes to be tested. */
+    private Collection<Class<? extends StrictRandomAccessOutEdges>>
         edgesClasses = Lists.newArrayList();
 
     @Before
@@ -46,22 +46,22 @@ public class TestStrictRandomAccessEdges {
     }
 
     /**
-     * Ensures that all {@link StrictRandomAccessVertexEdges} implementations
+     * Ensures that all {@link StrictRandomAccessOutEdges} implementations
      * correctly return edge values.
      */
     @Test
     public void testParallelEdges() {
-      for (Class<? extends StrictRandomAccessVertexEdges> edgesClass :
+      for (Class<? extends StrictRandomAccessOutEdges> edgesClass :
           edgesClasses) {
         testParallelEdgesClass(edgesClass);
       }
     }
 
     private void testParallelEdgesClass(
-        Class<? extends StrictRandomAccessVertexEdges> edgesClass) {
-      StrictRandomAccessVertexEdges<LongWritable, DoubleWritable> edges =
-          (StrictRandomAccessVertexEdges<LongWritable, DoubleWritable>)
-              instantiateVertexEdges(edgesClass);
+        Class<? extends StrictRandomAccessOutEdges> edgesClass) {
+      StrictRandomAccessOutEdges<LongWritable, DoubleWritable> edges =
+          (StrictRandomAccessOutEdges<LongWritable, DoubleWritable>)
+              instantiateOutEdges(edgesClass);
 
       // Initial edges list contains parallel edges.
       List<Edge<LongWritable, DoubleWritable>> initialEdges = Lists.newArrayList(

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
index 8a048fd..15d2bb0 100644
--- a/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
+++ b/giraph-core/src/test/java/org/apache/giraph/graph/TestVertexAndEdges.java
@@ -29,7 +29,7 @@ import org.apache.giraph.edge.HashMultimapEdges;
 import org.apache.giraph.edge.LongDoubleArrayEdges;
 import org.apache.giraph.edge.LongDoubleHashMapEdges;
 import org.apache.giraph.edge.MutableEdge;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.time.SystemTime;
 import org.apache.giraph.time.Time;
 import org.apache.giraph.time.Times;
@@ -57,14 +57,14 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 /**
- * Test {@link Vertex} functionality across the provided {@link VertexEdges}
+ * Test {@link Vertex} functionality across the provided {@link org.apache.giraph.edge.OutEdges}
  * classes.
  */
 public class TestVertexAndEdges {
   /** Number of repetitions. */
   public static final int REPS = 100;
-  /** {@link org.apache.giraph.edge.VertexEdges} classes to be tested. */
-  private Collection<Class<? extends VertexEdges>> edgesClasses =
+  /** {@link org.apache.giraph.edge.OutEdges} classes to be tested. */
+  private Collection<Class<? extends OutEdges>> edgesClasses =
       Lists.newArrayList();
 
   /**
@@ -77,12 +77,12 @@ public class TestVertexAndEdges {
   }
 
   /**
-   * A basic {@link VertexEdges} implementation that doesn't provide any
+   * A basic {@link org.apache.giraph.edge.OutEdges} implementation that doesn't provide any
    * special functionality. Used to test the default implementations of
    * Vertex#getEdgeValue(), Vertex#getMutableEdges(), etc.
    */
-  public static class TestVertexEdges
-      implements VertexEdges<LongWritable, DoubleWritable> {
+  public static class TestOutEdges
+      implements OutEdges<LongWritable, DoubleWritable> {
     private List<Edge<LongWritable, DoubleWritable>> edgeList;
 
 
@@ -151,7 +151,7 @@ public class TestVertexAndEdges {
 
   @Before
   public void setUp() {
-    edgesClasses.add(TestVertexEdges.class);
+    edgesClasses.add(TestOutEdges.class);
     edgesClasses.add(ByteArrayEdges.class);
     edgesClasses.add(ArrayListEdges.class);
     edgesClasses.add(HashMapEdges.class);
@@ -161,10 +161,10 @@ public class TestVertexAndEdges {
   }
 
   private Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
-  instantiateVertex(Class<? extends VertexEdges> edgesClass) {
+  instantiateVertex(Class<? extends OutEdges> edgesClass) {
     GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
     giraphConfiguration.setVertexClass(TestVertex.class);
-    giraphConfiguration.setVertexEdgesClass(edgesClass);
+    giraphConfiguration.setOutEdgesClass(edgesClass);
     ImmutableClassesGiraphConfiguration immutableClassesGiraphConfiguration =
         new ImmutableClassesGiraphConfiguration(giraphConfiguration);
     return immutableClassesGiraphConfiguration.createVertex();
@@ -185,34 +185,34 @@ public class TestVertexAndEdges {
     assertEquals(5.5f, vertex.getValue().get(), 0d);
   }
 
-  public static VertexEdges
-  instantiateVertexEdges(Class<? extends VertexEdges> edgesClass) {
+  public static OutEdges
+  instantiateOutEdges(Class<? extends OutEdges> edgesClass) {
     GiraphConfiguration giraphConfiguration = new GiraphConfiguration();
     // Needed to extract type arguments in ReflectionUtils.
     giraphConfiguration.setVertexClass(TestVertex.class);
-    giraphConfiguration.setVertexEdgesClass(edgesClass);
+    giraphConfiguration.setOutEdgesClass(edgesClass);
     ImmutableClassesGiraphConfiguration immutableClassesGiraphConfiguration =
         new ImmutableClassesGiraphConfiguration(giraphConfiguration);
-    return immutableClassesGiraphConfiguration.createVertexEdges();
+    return immutableClassesGiraphConfiguration.createOutEdges();
   }
 
   /**
-   * Test the provided {@link VertexEdges} implementations for instantiation,
+   * Test the provided {@link org.apache.giraph.edge.OutEdges} implementations for instantiation,
    * initialization, edge addition, and edge removal.
    */
   @Test
   public void testEdges() {
-    for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+    for (Class<? extends OutEdges> edgesClass : edgesClasses) {
       testEdgesClass(edgesClass);
     }
   }
 
-  private void testEdgesClass(Class<? extends VertexEdges> edgesClass) {
+  private void testEdgesClass(Class<? extends OutEdges> edgesClass) {
     Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
         instantiateVertex(edgesClass);
-    VertexEdges<LongWritable, DoubleWritable> vertexEdges =
-        instantiateVertexEdges(edgesClass);
-    assertNotNull(vertexEdges);
+    OutEdges<LongWritable, DoubleWritable> outEdges =
+        instantiateOutEdges(edgesClass);
+    assertNotNull(outEdges);
 
     List<Edge<LongWritable, DoubleWritable>> edges = Lists.newLinkedList();
     for (int i = 1000; i > 0; --i) {
@@ -220,8 +220,8 @@ public class TestVertexAndEdges {
           new DoubleWritable(i * 2.0)));
     }
 
-    vertexEdges.initialize(edges);
-    vertex.initialize(new LongWritable(1), new FloatWritable(1), vertexEdges);
+    outEdges.initialize(edges);
+    vertex.initialize(new LongWritable(1), new FloatWritable(1), outEdges);
 
     assertEquals(20.0, vertex.getEdgeValue(new LongWritable(10)).get(), 0.0);
 
@@ -246,19 +246,19 @@ public class TestVertexAndEdges {
    */
   @Test
   public void testMutateEdges() {
-    for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
+    for (Class<? extends OutEdges> edgesClass : edgesClasses) {
       testMutateEdgesClass(edgesClass);
     }
   }
 
-  private void testMutateEdgesClass(Class<? extends VertexEdges> edgesClass) {
+  private void testMutateEdgesClass(Class<? extends OutEdges> edgesClass) {
     Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
         instantiateVertex(edgesClass);
-    VertexEdges<LongWritable, DoubleWritable> vertexEdges =
-        instantiateVertexEdges(edgesClass);
+    OutEdges<LongWritable, DoubleWritable> outEdges =
+        instantiateOutEdges(edgesClass);
 
-    vertexEdges.initialize();
-    vertex.initialize(new LongWritable(0), new FloatWritable(0), vertexEdges);
+    outEdges.initialize();
+    vertex.initialize(new LongWritable(0), new FloatWritable(0), outEdges);
 
     // Add 10 edges with id i, value i for i = 0..9
     for (int i = 0; i < 10; ++i) {
@@ -327,24 +327,24 @@ public class TestVertexAndEdges {
   }
 
   /**
-   * Test {@link Vertex} and {@link VertexEdges} serialization.
+   * Test {@link Vertex} and {@link org.apache.giraph.edge.OutEdges} serialization.
    * @throws IOException
    */
   @Test
   public void testSerialize() throws IOException {
-    for (Class<? extends VertexEdges> edgesClass : edgesClasses) {
-      testSerializeVertexEdgesClass(edgesClass);
-      testDynamicChannelBufferSerializeVertexEdgesClass(edgesClass);
-      testUnsafeSerializeVertexEdgesClass(edgesClass);
+    for (Class<? extends OutEdges> edgesClass : edgesClasses) {
+      testSerializeOutEdgesClass(edgesClass);
+      testDynamicChannelBufferSerializeOutEdgesClass(edgesClass);
+      testUnsafeSerializeOutEdgesClass(edgesClass);
     }
   }
 
   private Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable>
-  buildVertex(Class<? extends VertexEdges> edgesClass) {
+  buildVertex(Class<? extends OutEdges> edgesClass) {
     Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
         instantiateVertex(edgesClass);
-    VertexEdges<LongWritable, DoubleWritable> vertexEdges =
-        instantiateVertexEdges(edgesClass);
+    OutEdges<LongWritable, DoubleWritable> outEdges =
+        instantiateOutEdges(edgesClass);
 
     int edgesCount = 200;
     List<Edge<LongWritable, DoubleWritable>> edges =
@@ -354,14 +354,14 @@ public class TestVertexAndEdges {
           new DoubleWritable(i * 2.0)));
     }
 
-    vertexEdges.initialize(edges);
+    outEdges.initialize(edges);
     vertex.initialize(new LongWritable(2), new FloatWritable(3.0f),
-        vertexEdges);
+        outEdges);
     return vertex;
   }
 
-  private void testSerializeVertexEdgesClass(
-      Class<? extends VertexEdges> edgesClass) {
+  private void testSerializeOutEdgesClass(
+      Class<? extends OutEdges> edgesClass) {
     Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
         buildVertex(edgesClass);
 
@@ -404,8 +404,8 @@ public class TestVertexAndEdges {
     assertTrue(EdgeIterables.sameEdges(vertex.getEdges(), readVertex.getEdges()));
   }
 
-  private void testDynamicChannelBufferSerializeVertexEdgesClass(
-      Class<? extends VertexEdges> edgesClass)
+  private void testDynamicChannelBufferSerializeOutEdgesClass(
+      Class<? extends OutEdges> edgesClass)
       throws IOException {
     Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
         buildVertex(edgesClass);
@@ -423,7 +423,7 @@ public class TestVertexAndEdges {
           serializeNanosStart);
     }
     serializeNanos /= REPS;
-    System.out.println("testDynamicChannelBufferSerializeVertexEdgesClass: " +
+    System.out.println("testDynamicChannelBufferSerializeOutEdgesClass: " +
         "Serializing took " + serializeNanos + " ns for " +
         outputStream.getDynamicChannelBuffer().writerIndex() + " bytes " +
         (outputStream.getDynamicChannelBuffer().writerIndex() * 1f *
@@ -447,7 +447,7 @@ public class TestVertexAndEdges {
       outputStream.getDynamicChannelBuffer().readerIndex(0);
     }
     deserializeNanos /= REPS;
-    System.out.println("testDynamicChannelBufferSerializeVertexEdgesClass: " +
+    System.out.println("testDynamicChannelBufferSerializeOutEdgesClass: " +
         "Deserializing took " + deserializeNanos + " ns for " +
         outputStream.getDynamicChannelBuffer().writerIndex() + " bytes " +
         (outputStream.getDynamicChannelBuffer().writerIndex() * 1f *
@@ -459,8 +459,8 @@ public class TestVertexAndEdges {
     assertTrue(EdgeIterables.sameEdges(vertex.getEdges(), readVertex.getEdges()));
   }
 
-  private void testUnsafeSerializeVertexEdgesClass(
-      Class<? extends VertexEdges> edgesClass)
+  private void testUnsafeSerializeOutEdgesClass(
+      Class<? extends OutEdges> edgesClass)
       throws IOException {
     Vertex<LongWritable, FloatWritable, DoubleWritable, LongWritable> vertex =
         buildVertex(edgesClass);
@@ -478,7 +478,7 @@ public class TestVertexAndEdges {
           serializeNanosStart);
     }
     serializeNanos /= REPS;
-    System.out.println("testUnsafeSerializeVertexEdgesClass: " +
+    System.out.println("testUnsafeSerializeOutEdgesClass: " +
         "Serializing took " +
         serializeNanos +
         " ns for " + outputStream.getPos()
@@ -503,7 +503,7 @@ public class TestVertexAndEdges {
           deserializeNanosStart);
     }
     deserializeNanos /= REPS;
-    System.out.println("testUnsafeSerializeVertexEdgesClass: " +
+    System.out.println("testUnsafeSerializeOutEdgesClass: " +
         "Deserializing took " +
         deserializeNanos +
         " ns for " + outputStream.getPos() +

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
index ede76f0..07d4cc8 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
@@ -65,7 +65,7 @@ public class TestEdgeInput extends BspCase {
 
     GiraphConfiguration conf = new GiraphConfiguration();
     conf.setVertexClass(TestVertexWithNumEdges.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> results = InternalVertexRunner.run(conf, null, edges);
@@ -94,7 +94,7 @@ public class TestEdgeInput extends BspCase {
 
     GiraphConfiguration conf = new GiraphConfiguration();
     conf.setVertexClass(TestVertexWithNumEdges.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setEdgeInputFormatClass(IntNullReverseTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> results = InternalVertexRunner.run(conf, null, edges);
@@ -130,7 +130,7 @@ public class TestEdgeInput extends BspCase {
 
     GiraphConfiguration conf = new GiraphConfiguration();
     conf.setVertexClass(TestVertexDoNothing.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(IntIntTextVertexValueInputFormat.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
@@ -161,7 +161,7 @@ public class TestEdgeInput extends BspCase {
 
     conf = new GiraphConfiguration();
     conf.setVertexClass(TestVertexWithNumEdges.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(IntIntTextVertexValueInputFormat.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
@@ -179,7 +179,7 @@ public class TestEdgeInput extends BspCase {
     assertEquals(1, (int) values.get(5));
   }
 
-  // It should use the specified input VertexEdges class.
+  // It should use the specified input OutEdges class.
   @Test
   public void testDifferentInputEdgesClass() throws Exception {
     String[] edges = new String[] {
@@ -191,8 +191,8 @@ public class TestEdgeInput extends BspCase {
 
     GiraphConfiguration conf = new GiraphConfiguration();
     conf.setVertexClass(TestVertexCheckEdgesType.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
-    conf.setInputVertexEdgesClass(TestVertexEdgesFilterEven.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
+    conf.setInputOutEdgesClass(TestOutEdgesFilterEven.class);
     conf.setEdgeInputFormatClass(IntNullTextEdgeInputFormat.class);
     conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);
     Iterable<String> results = InternalVertexRunner.run(conf, null, edges);
@@ -221,7 +221,7 @@ public class TestEdgeInput extends BspCase {
   public static class TestVertexCheckEdgesType extends TestVertexWithNumEdges {
     @Override
     public void compute(Iterable<NullWritable> messages) throws IOException {
-      assertFalse(getEdges() instanceof TestVertexEdgesFilterEven);
+      assertFalse(getEdges() instanceof TestOutEdgesFilterEven);
       assertTrue(getEdges() instanceof ByteArrayEdges);
       super.compute(messages);
     }
@@ -247,7 +247,7 @@ public class TestEdgeInput extends BspCase {
     }
   }
 
-  public static class TestVertexEdgesFilterEven
+  public static class TestOutEdgesFilterEven
       extends ByteArrayEdges<IntWritable, NullWritable> {
     @Override
     public void add(Edge<IntWritable, NullWritable> edge) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 1c34f78..9d6b215 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
@@ -89,7 +89,7 @@ public class TestMasterObserver {
     conf.set(GiraphConstants.MASTER_OBSERVER_CLASSES.getKey(),
         arrayToString(klasses));
     conf.setVertexClass(NoOpVertex.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(IntNullNullTextInputFormat.class);
     InternalVertexRunner.run(conf, graph);
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 bdcc49e..9b655af 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
@@ -19,7 +19,7 @@ package org.apache.giraph.partition;
 
 import org.apache.giraph.edge.ArrayListEdges;
 import org.apache.giraph.edge.EdgeFactory;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.GiraphTransferRegulator;
 import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.graph.Vertex;
@@ -71,7 +71,7 @@ public class TestGiraphTransferRegulator {
         .setInt(GiraphTransferRegulator.MAX_VERTICES_PER_TRANSFER, 1);
     job.getConfiguration()
         .setInt(GiraphTransferRegulator.MAX_EDGES_PER_TRANSFER, 3);
-    VertexEdges<IntWritable, DoubleWritable> edges =
+    OutEdges<IntWritable, DoubleWritable> edges =
         new ArrayListEdges<IntWritable, DoubleWritable>();
     edges.initialize(3);
     edges.add(EdgeFactory.create(new IntWritable(2), new DoubleWritable(22)));

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 5d3fae1..5d71259 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
@@ -67,7 +67,7 @@ public class ConnectedComponentsVertexTest {
 
         GiraphConfiguration conf = new GiraphConfiguration();
         conf.setVertexClass(ConnectedComponentsVertex.class);
-        conf.setVertexEdgesClass(ByteArrayEdges.class);
+        conf.setOutEdgesClass(ByteArrayEdges.class);
         conf.setCombinerClass(MinimumIntCombiner.class);
         conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
         conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 f748cb0..e4c029c 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
@@ -58,7 +58,7 @@ public class ConnectedComponentsVertexTestInMemory {
   public void testToyData() throws Exception {
     GiraphConfiguration conf = new GiraphConfiguration();
     conf.setVertexClass(ConnectedComponentsVertex.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setCombinerClass(MinimumIntCombiner.class);
 
     TestGraph<IntWritable, IntWritable, NullWritable, IntWritable> graph =

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 b41bcf6..2a39291 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
@@ -51,7 +51,7 @@ public class PageRankVertexTest {
     conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 50);
     conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.15f);
     conf.setVertexClass(PageRankVertex.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(LongDoubleNullTextInputFormat.class);
     conf.setVertexOutputFormatClass(
         VertexWithDoubleValueNullEdgeTextOutputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 6ecfefe..a2a891e 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
@@ -45,7 +45,7 @@ public class RandomWalkWithRestartVertexTest {
     conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 30);
     conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.25f);
     conf.setVertexClass(RandomWalkWithRestartVertex.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(LongDoubleDoubleTextInputFormat.class);
     conf.setVertexOutputFormatClass(
         VertexWithDoubleValueDoubleEdgeTextOutputFormat.class);
@@ -81,7 +81,7 @@ public class RandomWalkWithRestartVertexTest {
     conf.setInt(RandomWalkWithRestartVertex.MAX_SUPERSTEPS, 30);
     conf.setFloat(RandomWalkWithRestartVertex.TELEPORTATION_PROBABILITY, 0.15f);
     conf.setVertexClass(RandomWalkWithRestartVertex.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(
         NormalizingLongDoubleDoubleTextInputFormat.class);
     conf.setVertexOutputFormatClass(

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 ee99dc5..1006577 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
@@ -124,7 +124,7 @@ public class SimpleShortestPathsVertexTest {
     // start from vertex 1
     SOURCE_ID.set(conf, 1);
     conf.setVertexClass(SimpleShortestPathsVertex.class);
-    conf.setVertexEdgesClass(ByteArrayEdges.class);
+    conf.setOutEdgesClass(ByteArrayEdges.class);
     conf.setVertexInputFormatClass(
         JsonLongDoubleFloatDoubleVertexInputFormat.class);
     conf.setVertexOutputFormatClass(

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 4945b52..23d055e 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
@@ -71,7 +71,7 @@ public class TryMultiIpcBindingPortsTest {
         GiraphConfiguration conf = new GiraphConfiguration();
         GiraphConstants.FAIL_FIRST_IPC_PORT_BIND_ATTEMPT.set(conf, true);
         conf.setVertexClass(ConnectedComponentsVertex.class);
-        conf.setVertexEdgesClass(ByteArrayEdges.class);
+        conf.setOutEdgesClass(ByteArrayEdges.class);
         conf.setCombinerClass(MinimumIntCombiner.class);
         conf.setVertexInputFormatClass(IntIntNullTextInputFormat.class);
         conf.setVertexOutputFormatClass(IdWithValueTextOutputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/src/site/xdoc/io.xml
----------------------------------------------------------------------
diff --git a/src/site/xdoc/io.xml b/src/site/xdoc/io.xml
index cb752cb..76f610b 100644
--- a/src/site/xdoc/io.xml
+++ b/src/site/xdoc/io.xml
@@ -1,23 +1,22 @@
 <?xml version="1.0" encoding="UTF-8"?>
 
 <!--
-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.
--->
+  ~ 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.
+  -->
 
 <document xmlns="http://maven.apache.org/XDOC/2.0"
 	  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"


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

Posted by ap...@apache.org.
GIRAPH-633: Rename VertexEdges to OutEdges (apresta)


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

Branch: refs/heads/trunk
Commit: 158c0b1180fa8f01bffd119df49b982fdda22771
Parents: 38f1f17
Author: Alessandro Presta <al...@fb.com>
Authored: Mon Apr 15 11:40:47 2013 -0700
Committer: Alessandro Presta <al...@fb.com>
Committed: Mon Apr 15 13:53:47 2013 -0700

----------------------------------------------------------------------
 .../apache/giraph/benchmark/PageRankBenchmark.java |    2 +-
 .../giraph/benchmark/ShortestPathsBenchmark.java   |    6 +-
 .../benchmark/WeightedPageRankBenchmark.java       |   16 ++--
 .../java/org/apache/giraph/conf/GiraphClasses.java |   48 ++++----
 .../apache/giraph/conf/GiraphConfiguration.java    |   34 +++---
 .../org/apache/giraph/conf/GiraphConstants.java    |   14 +-
 .../conf/ImmutableClassesGiraphConfiguration.java  |   95 ++++++++-------
 .../org/apache/giraph/edge/ArrayListEdges.java     |    6 +-
 .../org/apache/giraph/edge/ByteArrayEdges.java     |    6 +-
 .../apache/giraph/edge/ConfigurableOutEdges.java   |   36 ++++++
 .../giraph/edge/ConfigurableVertexEdges.java       |   36 ------
 .../java/org/apache/giraph/edge/EdgeStore.java     |   52 ++++----
 .../java/org/apache/giraph/edge/HashMapEdges.java  |    8 +-
 .../org/apache/giraph/edge/HashMultimapEdges.java  |    6 +-
 .../org/apache/giraph/edge/IntNullArrayEdges.java  |    4 +-
 .../apache/giraph/edge/LongDoubleArrayEdges.java   |    6 +-
 .../apache/giraph/edge/LongDoubleHashMapEdges.java |    8 +-
 .../org/apache/giraph/edge/LongNullArrayEdges.java |    6 +-
 .../apache/giraph/edge/LongNullHashSetEdges.java   |    6 +-
 .../giraph/edge/MultiRandomAccessOutEdges.java     |   41 ++++++
 .../giraph/edge/MultiRandomAccessVertexEdges.java  |   41 ------
 .../apache/giraph/edge/MutableEdgesIterable.java   |    6 +-
 .../apache/giraph/edge/MutableEdgesWrapper.java    |   24 ++--
 .../org/apache/giraph/edge/MutableOutEdges.java    |   42 +++++++
 .../org/apache/giraph/edge/MutableVertexEdges.java |   42 -------
 .../main/java/org/apache/giraph/edge/OutEdges.java |   84 +++++++++++++
 .../apache/giraph/edge/ReuseObjectsOutEdges.java   |   34 +++++
 .../giraph/edge/ReuseObjectsVertexEdges.java       |   34 -----
 .../giraph/edge/StrictRandomAccessOutEdges.java    |   51 ++++++++
 .../giraph/edge/StrictRandomAccessVertexEdges.java |   51 --------
 .../java/org/apache/giraph/edge/VertexEdges.java   |   84 -------------
 .../main/java/org/apache/giraph/graph/Vertex.java  |   55 ++++----
 .../io/formats/PseudoRandomEdgeInputFormat.java    |   12 +-
 .../PseudoRandomIntNullVertexInputFormat.java      |    6 +-
 .../io/formats/PseudoRandomVertexInputFormat.java  |    8 +-
 .../giraph/job/GiraphConfigurationValidator.java   |   34 +++---
 .../giraph/partition/ByteArrayPartition.java       |    4 +-
 .../giraph/partition/DiskBackedPartitionStore.java |   16 ++--
 .../apache/giraph/utils/ConfigurationUtils.java    |   12 +-
 .../org/apache/giraph/utils/WritableUtils.java     |    8 +-
 .../apache/giraph/edge/TestMultiGraphEdges.java    |   18 ++--
 .../giraph/edge/TestMultiRandomAccessEdges.java    |   20 ++--
 .../org/apache/giraph/edge/TestNullValueEdges.java |   28 ++--
 .../apache/giraph/edge/TestStrictGraphEdges.java   |   18 ++--
 .../giraph/edge/TestStrictRandomAccessEdges.java   |   20 ++--
 .../apache/giraph/graph/TestVertexAndEdges.java    |   96 +++++++-------
 .../java/org/apache/giraph/io/TestEdgeInput.java   |   18 ++--
 .../apache/giraph/master/TestMasterObserver.java   |    2 +-
 .../partition/TestGiraphTransferRegulator.java     |    4 +-
 .../examples/ConnectedComponentsVertexTest.java    |    2 +-
 .../ConnectedComponentsVertexTestInMemory.java     |    2 +-
 .../apache/giraph/examples/PageRankVertexTest.java |    2 +-
 .../examples/RandomWalkWithRestartVertexTest.java  |    4 +-
 .../examples/SimpleShortestPathsVertexTest.java    |    2 +-
 .../examples/TryMultiIpcBindingPortsTest.java      |    2 +-
 src/site/xdoc/io.xml                               |   33 +++---
 56 files changed, 679 insertions(+), 676 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
index 3a755c8..0f8d284 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
@@ -45,7 +45,7 @@ public class PageRankBenchmark extends GiraphBenchmark {
   protected void prepareConfiguration(GiraphConfiguration conf,
       CommandLine cmd) {
     conf.setVertexClass(PageRankVertex.class);
-    conf.setVertexEdgesClass(IntNullArrayEdges.class);
+    conf.setOutEdgesClass(IntNullArrayEdges.class);
     conf.setCombinerClass(FloatSumCombiner.class);
     conf.setVertexInputFormatClass(
         PseudoRandomIntNullVertexInputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
index df53ee6..8e6c877 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
@@ -41,7 +41,7 @@ public class ShortestPathsBenchmark extends GiraphBenchmark {
   private static final Logger LOG =
       Logger.getLogger(ShortestPathsBenchmark.class);
 
-  /** Option for VertexEdges class */
+  /** Option for OutEdges class */
   private static final BenchmarkOption EDGES_CLASS = new BenchmarkOption(
       "c", "edgesClass", true,
       "Vertex edges class (0 for HashMapEdges, 1 for ArrayListEdges)");
@@ -60,9 +60,9 @@ public class ShortestPathsBenchmark extends GiraphBenchmark {
       CommandLine cmd) {
     conf.setVertexClass(ShortestPathsVertex.class);
     if (EDGES_CLASS.getOptionIntValue(cmd, 1) == 1) {
-      conf.setVertexEdgesClass(ArrayListEdges.class);
+      conf.setOutEdgesClass(ArrayListEdges.class);
     } else {
-      conf.setVertexEdgesClass(HashMapEdges.class);
+      conf.setOutEdgesClass(HashMapEdges.class);
     }
     LOG.info("Using class " + GiraphConstants.VERTEX_CLASS.get(conf));
     conf.setVertexInputFormatClass(PseudoRandomVertexInputFormat.class);

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
index 9de10b5..3fc514a 100644
--- a/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
+++ b/giraph-core/src/main/java/org/apache/giraph/benchmark/WeightedPageRankBenchmark.java
@@ -45,7 +45,7 @@ public class WeightedPageRankBenchmark extends GiraphBenchmark {
   private static final Logger LOG =
       Logger.getLogger(WeightedPageRankBenchmark.class);
 
-  /** Option for VertexEdges class */
+  /** Option for OutEdges class */
   private static final BenchmarkOption EDGES_CLASS = new BenchmarkOption(
       "c", "edgesClass", true,
       "Vertex edges class (0 for LongDoubleArrayEdges," +
@@ -92,25 +92,25 @@ public class WeightedPageRankBenchmark extends GiraphBenchmark {
     int edgesClassOption = EDGES_CLASS.getOptionIntValue(cmd, 1);
     switch (edgesClassOption) {
     case 0:
-      configuration.setVertexEdgesClass(LongDoubleArrayEdges.class);
+      configuration.setOutEdgesClass(LongDoubleArrayEdges.class);
       break;
     case 1:
-      configuration.setVertexEdgesClass(ByteArrayEdges.class);
+      configuration.setOutEdgesClass(ByteArrayEdges.class);
       break;
     case 2:
-      configuration.setVertexEdgesClass(ByteArrayEdges.class);
+      configuration.setOutEdgesClass(ByteArrayEdges.class);
       configuration.useUnsafeSerialization(true);
       break;
     case 3:
-      configuration.setVertexEdgesClass(ArrayListEdges.class);
+      configuration.setOutEdgesClass(ArrayListEdges.class);
       break;
     case 4:
-      configuration.setVertexEdgesClass(HashMapEdges.class);
+      configuration.setOutEdgesClass(HashMapEdges.class);
       break;
     default:
-      LOG.info("Unknown VertexEdges class, " +
+      LOG.info("Unknown OutEdges class, " +
           "defaulting to LongDoubleArrayEdges");
-      configuration.setVertexEdgesClass(LongDoubleArrayEdges.class);
+      configuration.setOutEdgesClass(LongDoubleArrayEdges.class);
     }
 
     LOG.info("Using edges class " +

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 95499bd..4a0e8f7 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
@@ -21,7 +21,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.aggregators.TextAggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
 import org.apache.giraph.graph.DefaultVertexValueFactory;
 import org.apache.giraph.graph.Vertex;
@@ -70,9 +70,9 @@ public class GiraphClasses<I extends WritableComparable,
   /** Message value class - cached for fast access */
   protected Class<M> messageValueClass;
   /** Vertex edges class - cached for fast access */
-  protected Class<? extends VertexEdges<I, E>> vertexEdgesClass;
+  protected Class<? extends OutEdges<I, E>> outEdgesClass;
   /** Input vertex edges class - cached for fast access */
-  protected Class<? extends VertexEdges<I, E>> inputVertexEdgesClass;
+  protected Class<? extends OutEdges<I, E>> inputOutEdgesClass;
 
   /** Vertex value factory class - cached for fast access */
   protected Class<? extends VertexValueFactory<V>> vertexValueFactoryClass;
@@ -114,9 +114,9 @@ public class GiraphClasses<I extends WritableComparable,
   public GiraphClasses() {
     // Note: the cast to Object is required in order for javac to accept the
     // downcast.
-    vertexEdgesClass = (Class<? extends VertexEdges<I, E>>) (Object)
+    outEdgesClass = (Class<? extends OutEdges<I, E>>) (Object)
         ByteArrayEdges.class;
-    inputVertexEdgesClass = (Class<? extends VertexEdges<I, E>>) (Object)
+    inputOutEdgesClass = (Class<? extends OutEdges<I, E>>) (Object)
         ByteArrayEdges.class;
     vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>) (Object)
         DefaultVertexValueFactory.class;
@@ -157,10 +157,10 @@ public class GiraphClasses<I extends WritableComparable,
     edgeValueClass = (Class<E>) classList.get(2);
     messageValueClass = (Class<M>) classList.get(3);
 
-    vertexEdgesClass = (Class<? extends VertexEdges<I, E>>)
+    outEdgesClass = (Class<? extends OutEdges<I, E>>)
         VERTEX_EDGES_CLASS.get(conf);
-    inputVertexEdgesClass = (Class<? extends VertexEdges<I, E>>)
-        INPUT_VERTEX_EDGES_CLASS.getWithDefault(conf, vertexEdgesClass);
+    inputOutEdgesClass = (Class<? extends OutEdges<I, E>>)
+        INPUT_VERTEX_EDGES_CLASS.getWithDefault(conf, outEdgesClass);
     vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>)
         VERTEX_VALUE_FACTORY_CLASS.get(conf);
 
@@ -237,16 +237,16 @@ public class GiraphClasses<I extends WritableComparable,
    *
    * @return Vertex edges class.
    */
-  public Class<? extends VertexEdges<I, E>> getVertexEdgesClass() {
-    return vertexEdgesClass;
+  public Class<? extends OutEdges<I, E>> getOutEdgesClass() {
+    return outEdgesClass;
   }
 
   /* Get Vertex edges class used during edge-based input
  *
  * @return Vertex edges class.
  */
-  public Class<? extends VertexEdges<I, E>> getInputVertexEdgesClass() {
-    return inputVertexEdgesClass;
+  public Class<? extends OutEdges<I, E>> getInputOutEdgesClass() {
+    return inputOutEdgesClass;
   }
 
 
@@ -508,29 +508,29 @@ public class GiraphClasses<I extends WritableComparable,
   }
 
   /**
-   * Set VertexEdges class held
+   * Set OutEdges class held
    *
-   * @param vertexEdgesClass Vertex edges class to set
+   * @param outEdgesClass Vertex edges class to set
    * @return this
    */
-  public GiraphClasses setVertexEdgesClass(
-      Class<? extends VertexEdges> vertexEdgesClass) {
-    this.vertexEdgesClass =
-        (Class<? extends VertexEdges<I, E>>) vertexEdgesClass;
+  public GiraphClasses setOutEdgesClass(
+      Class<? extends OutEdges> outEdgesClass) {
+    this.outEdgesClass =
+        (Class<? extends OutEdges<I, E>>) outEdgesClass;
     return this;
   }
 
   /**
-   * Set VertexEdges class used during edge-input (if different from the one
+   * Set OutEdges class used during edge-input (if different from the one
    * used for computation)
    *
-   * @param inputVertexEdgesClass Input vertex edges class to set
+   * @param inputOutEdgesClass Input vertex edges class to set
    * @return this
    */
-  public GiraphClasses setInputVertexEdgesClass(
-      Class<? extends VertexEdges> inputVertexEdgesClass) {
-    this.inputVertexEdgesClass =
-        (Class<? extends VertexEdges<I, E>>) inputVertexEdgesClass;
+  public GiraphClasses setInputOutEdgesClass(
+      Class<? extends OutEdges> inputOutEdgesClass) {
+    this.inputOutEdgesClass =
+        (Class<? extends OutEdges<I, E>>) inputOutEdgesClass;
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
index c527595..7f9e38e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
@@ -20,8 +20,8 @@ package org.apache.giraph.conf;
 
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
-import org.apache.giraph.edge.ReuseObjectsVertexEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.edge.ReuseObjectsOutEdges;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.graph.VertexValueFactory;
@@ -100,36 +100,36 @@ public class GiraphConfiguration extends Configuration
    *
    * @return vertex edges class
    */
-  public Class<? extends VertexEdges> getVertexEdgesClass() {
+  public Class<? extends OutEdges> getOutEdgesClass() {
     return VERTEX_EDGES_CLASS.get(this);
   }
 
   /**
    * Set the vertex edges class
    *
-   * @param vertexEdgesClass Determines the way edges are stored
+   * @param outEdgesClass Determines the way edges are stored
    */
-  public final void setVertexEdgesClass(
-      Class<? extends VertexEdges> vertexEdgesClass) {
-    VERTEX_EDGES_CLASS.set(this, vertexEdgesClass);
+  public final void setOutEdgesClass(
+      Class<? extends OutEdges> outEdgesClass) {
+    VERTEX_EDGES_CLASS.set(this, outEdgesClass);
   }
 
   /**
    * Set the vertex edges class used during edge-based input (if different
    * from the one used during computation)
    *
-   * @param inputVertexEdgesClass Determines the way edges are stored
+   * @param inputOutEdgesClass Determines the way edges are stored
    */
-  public final void setInputVertexEdgesClass(
-      Class<? extends VertexEdges> inputVertexEdgesClass) {
-    INPUT_VERTEX_EDGES_CLASS.set(this, inputVertexEdgesClass);
+  public final void setInputOutEdgesClass(
+      Class<? extends OutEdges> inputOutEdgesClass) {
+    INPUT_VERTEX_EDGES_CLASS.set(this, inputOutEdgesClass);
   }
 
   /**
-   * True if the {@link VertexEdges} implementation copies the passed edges
-   * to its own data structure, i.e. it doesn't keep references to Edge
-   * objects, target vertex ids or edge values passed to add() or
-   * initialize().
+   * True if the {@link org.apache.giraph.edge.OutEdges} implementation
+   * copies the passed edges to its own data structure,
+   * i.e. it doesn't keep references to Edge objects, target vertex ids or edge
+   * values passed to add() or initialize().
    * This makes it possible to reuse edge objects passed to the data
    * structure, to minimize object instantiation (see for example
    * EdgeStore#addPartitionEdges()).
@@ -137,8 +137,8 @@ public class GiraphConfiguration extends Configuration
    * @return True iff we can reuse the edge objects
    */
   public boolean reuseEdgeObjects() {
-    return ReuseObjectsVertexEdges.class.isAssignableFrom(
-        getVertexEdgesClass());
+    return ReuseObjectsOutEdges.class.isAssignableFrom(
+        getOutEdgesClass());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
index a55a2a2..6a5949e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
@@ -21,7 +21,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.aggregators.TextAggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
 import org.apache.giraph.graph.DefaultVertexValueFactory;
 import org.apache.giraph.graph.Vertex;
@@ -66,13 +66,13 @@ public interface GiraphConstants {
       ClassConfOption.create("giraph.vertexValueFactoryClass",
           DefaultVertexValueFactory.class, VertexValueFactory.class);
   /** Vertex edges class - optional */
-  ClassConfOption<VertexEdges> VERTEX_EDGES_CLASS =
-      ClassConfOption.create("giraph.vertexEdgesClass", ByteArrayEdges.class,
-          VertexEdges.class);
+  ClassConfOption<OutEdges> VERTEX_EDGES_CLASS =
+      ClassConfOption.create("giraph.outEdgesClass", ByteArrayEdges.class,
+          OutEdges.class);
   /** Vertex edges class to be used during edge input only - optional */
-  ClassConfOption<VertexEdges> INPUT_VERTEX_EDGES_CLASS =
-      ClassConfOption.create("giraph.inputVertexEdgesClass",
-          ByteArrayEdges.class, VertexEdges.class);
+  ClassConfOption<OutEdges> INPUT_VERTEX_EDGES_CLASS =
+      ClassConfOption.create("giraph.inputOutEdgesClass",
+          ByteArrayEdges.class, OutEdges.class);
 
   /** Class for Master - optional */
   ClassConfOption<MasterCompute> MASTER_COMPUTE_CLASS =

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 dc3a26f..8dfe546 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
@@ -22,8 +22,8 @@ import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.edge.ReusableEdge;
-import org.apache.giraph.edge.VertexEdges;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
@@ -588,96 +588,99 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
   }
 
   @Override
-  public Class<? extends VertexEdges<I, E>> getVertexEdgesClass() {
-    return classes.getVertexEdgesClass();
+  public Class<? extends OutEdges<I, E>> getOutEdgesClass() {
+    return classes.getOutEdgesClass();
   }
 
   /**
-   * Get the user's subclassed {@link VertexEdges} used for input
+   * Get the user's subclassed {@link org.apache.giraph.edge.OutEdges} used for
+   * input
    *
    * @return User's input vertex edges class
    */
-  public Class<? extends VertexEdges<I, E>> getInputVertexEdgesClass() {
-    return classes.getInputVertexEdgesClass();
+  public Class<? extends OutEdges<I, E>> getInputOutEdgesClass() {
+    return classes.getInputOutEdgesClass();
   }
 
   /**
-   * Check whether the user has specified a different {@link VertexEdges}
-   * class to be used during edge-based input.
+   * Check whether the user has specified a different
+   * {@link org.apache.giraph.edge.OutEdges} class to be used during
+   * edge-based input.
    *
    * @return True iff there is a special edges class for input
    */
-  public boolean useInputVertexEdges() {
-    return classes.getInputVertexEdgesClass() != classes.getVertexEdgesClass();
+  public boolean useInputOutEdges() {
+    return classes.getInputOutEdgesClass() != classes.getOutEdgesClass();
   }
 
   /**
-   * Create a user {@link VertexEdges}
+   * Create a user {@link org.apache.giraph.edge.OutEdges}
    *
-   * @return Instantiated user VertexEdges
+   * @return Instantiated user OutEdges
    */
-  public VertexEdges<I, E> createVertexEdges() {
-    return ReflectionUtils.newInstance(getVertexEdgesClass(), this);
+  public OutEdges<I, E> createOutEdges() {
+    return ReflectionUtils.newInstance(getOutEdgesClass(), this);
   }
 
   /**
-   * Create a {@link VertexEdges} instance and initialize it with the default
-   * capacity.
+   * Create a {@link org.apache.giraph.edge.OutEdges} instance and initialize
+   * it with the default capacity.
    *
-   * @return Instantiated VertexEdges
+   * @return Instantiated OutEdges
    */
-  public VertexEdges<I, E> createAndInitializeVertexEdges() {
-    VertexEdges<I, E> vertexEdges = createVertexEdges();
-    vertexEdges.initialize();
-    return vertexEdges;
+  public OutEdges<I, E> createAndInitializeOutEdges() {
+    OutEdges<I, E> outEdges = createOutEdges();
+    outEdges.initialize();
+    return outEdges;
   }
 
   /**
-   * Create a {@link VertexEdges} instance and initialize it with the given
-   * capacity (the number of edges that will be added).
+   * Create a {@link org.apache.giraph.edge.OutEdges} instance and initialize
+   * it with the given capacity (the number of edges that will be added).
    *
    * @param capacity Number of edges that will be added
-   * @return Instantiated VertexEdges
+   * @return Instantiated OutEdges
    */
-  public VertexEdges<I, E> createAndInitializeVertexEdges(int capacity) {
-    VertexEdges<I, E> vertexEdges = createVertexEdges();
-    vertexEdges.initialize(capacity);
-    return vertexEdges;
+  public OutEdges<I, E> createAndInitializeOutEdges(int capacity) {
+    OutEdges<I, E> outEdges = createOutEdges();
+    outEdges.initialize(capacity);
+    return outEdges;
   }
 
   /**
-   * Create a {@link VertexEdges} instance and initialize it with the given
-   * iterable of edges.
+   * Create a {@link org.apache.giraph.edge.OutEdges} instance and initialize
+   * it with the given iterable of edges.
    *
    * @param edges Iterable of edges to add
-   * @return Instantiated VertexEdges
+   * @return Instantiated OutEdges
    */
-  public VertexEdges<I, E> createAndInitializeVertexEdges(
+  public OutEdges<I, E> createAndInitializeOutEdges(
       Iterable<Edge<I, E>> edges) {
-    VertexEdges<I, E> vertexEdges = createVertexEdges();
-    vertexEdges.initialize(edges);
-    return vertexEdges;
+    OutEdges<I, E> outEdges = createOutEdges();
+    outEdges.initialize(edges);
+    return outEdges;
   }
 
   /**
-   * Create a user {@link VertexEdges} used during edge-based input
+   * Create a user {@link org.apache.giraph.edge.OutEdges} used during
+   * edge-based input
    *
-   * @return Instantiated user input VertexEdges
+   * @return Instantiated user input OutEdges
    */
-  public VertexEdges<I, E> createInputVertexEdges() {
-    return ReflectionUtils.newInstance(getInputVertexEdgesClass(), this);
+  public OutEdges<I, E> createInputOutEdges() {
+    return ReflectionUtils.newInstance(getInputOutEdgesClass(), this);
   }
 
   /**
-   * Create an input {@link VertexEdges} instance and initialize it with the
-   * default capacity.
+   * Create an input {@link org.apache.giraph.edge.OutEdges} instance and
+   * initialize it with the default capacity.
    *
-   * @return Instantiated input VertexEdges
+   * @return Instantiated input OutEdges
    */
-  public VertexEdges<I, E> createAndInitializeInputVertexEdges() {
-    VertexEdges<I, E> vertexEdges = createInputVertexEdges();
-    vertexEdges.initialize();
-    return vertexEdges;
+  public OutEdges<I, E> createAndInitializeInputOutEdges() {
+    OutEdges<I, E> outEdges = createInputOutEdges();
+    outEdges.initialize();
+    return outEdges;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
index dda7568..4eb3378 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ArrayListEdges.java
@@ -30,15 +30,15 @@ import java.util.ArrayList;
 import java.util.Iterator;
 
 /**
- * {@link VertexEdges} implementation backed by an {@link ArrayList}.
+ * {@link OutEdges} implementation backed by an {@link ArrayList}.
  * Parallel edges are allowed.
  *
  * @param <I> Vertex id
  * @param <E> Edge value
  */
 public class ArrayListEdges<I extends WritableComparable, E extends Writable>
-    extends ConfigurableVertexEdges<I, E>
-    implements MutableVertexEdges<I, E> {
+    extends ConfigurableOutEdges<I, E>
+    implements MutableOutEdges<I, E> {
   /** List of edges. */
   private ArrayList<Edge<I, E>> edgeList;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
index 16748ef..b9d1f6c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ByteArrayEdges.java
@@ -33,7 +33,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 /**
- * {@link VertexEdges} implementation backed by a byte array.
+ * {@link OutEdges} implementation backed by a byte array.
  * Parallel edges are allowed.
  * Note: this implementation is optimized for space usage,
  * but edge removals are expensive.
@@ -42,8 +42,8 @@ import java.util.List;
  * @param <E> Edge value
  */
 public class ByteArrayEdges<I extends WritableComparable, E extends Writable>
-    extends ConfigurableVertexEdges<I, E>
-    implements ReuseObjectsVertexEdges<I, E> {
+    extends ConfigurableOutEdges<I, E>
+    implements ReuseObjectsOutEdges<I, E> {
   /** Serialized edges. */
   private byte[] serializedEdges;
   /** Number of bytes used in serializedEdges. */

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
new file mode 100644
index 0000000..631b209
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableOutEdges.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.edge;
+
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Common base class for {@link OutEdges} implementations that require a
+ * configuration.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public abstract class ConfigurableOutEdges<I extends WritableComparable,
+    E extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E,
+    Writable> implements OutEdges<I, E> {
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java
deleted file mode 100644
index f05e0d7..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ConfigurableVertexEdges.java
+++ /dev/null
@@ -1,36 +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.edge;
-
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Common base class for {@link VertexEdges} implementations that require a
- * configuration.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public abstract class ConfigurableVertexEdges<I extends WritableComparable,
-    E extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, Writable, E,
-    Writable> implements VertexEdges<I, E> {
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java b/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
index e8cb620..81044bd 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/EdgeStore.java
@@ -55,17 +55,17 @@ public class EdgeStore<I extends WritableComparable,
   private Progressable progressable;
   /** Map used to temporarily store incoming edges. */
   private ConcurrentMap<Integer,
-      ConcurrentMap<I, VertexEdges<I, E>>> transientEdges;
+      ConcurrentMap<I, OutEdges<I, E>>> transientEdges;
   /**
-   * Whether the chosen {@link VertexEdges} implementation allows for Edge
+   * Whether the chosen {@link OutEdges} implementation allows for Edge
    * reuse.
    */
   private boolean reuseEdgeObjects;
   /**
-   * Whether the {@link VertexEdges} class used during input is different
+   * Whether the {@link OutEdges} class used during input is different
    * from the one used during computation.
    */
-  private boolean useInputVertexEdges;
+  private boolean useInputOutEdges;
 
   /**
    * Constructor.
@@ -84,7 +84,7 @@ public class EdgeStore<I extends WritableComparable,
     transientEdges = new MapMaker().concurrencyLevel(
         configuration.getNettyServerExecutionConcurrency()).makeMap();
     reuseEdgeObjects = configuration.reuseEdgeObjects();
-    useInputVertexEdges = configuration.useInputVertexEdges();
+    useInputOutEdges = configuration.useInputOutEdges();
   }
 
   /**
@@ -96,10 +96,10 @@ public class EdgeStore<I extends WritableComparable,
    */
   public void addPartitionEdges(
       int partitionId, ByteArrayVertexIdEdges<I, E> edges) {
-    ConcurrentMap<I, VertexEdges<I, E>> partitionEdges =
+    ConcurrentMap<I, OutEdges<I, E>> partitionEdges =
         transientEdges.get(partitionId);
     if (partitionEdges == null) {
-      ConcurrentMap<I, VertexEdges<I, E>> newPartitionEdges =
+      ConcurrentMap<I, OutEdges<I, E>> newPartitionEdges =
           new MapMaker().concurrencyLevel(
               configuration.getNettyServerExecutionConcurrency()).makeMap();
       partitionEdges = transientEdges.putIfAbsent(partitionId,
@@ -116,38 +116,38 @@ public class EdgeStore<I extends WritableComparable,
       Edge<I, E> edge = reuseEdgeObjects ?
           vertexIdEdgeIterator.getCurrentEdge() :
           vertexIdEdgeIterator.releaseCurrentEdge();
-      VertexEdges<I, E> vertexEdges = partitionEdges.get(vertexId);
-      if (vertexEdges == null) {
-        VertexEdges<I, E> newVertexEdges =
-            configuration.createAndInitializeInputVertexEdges();
-        vertexEdges = partitionEdges.putIfAbsent(vertexId, newVertexEdges);
-        if (vertexEdges == null) {
-          vertexEdges = newVertexEdges;
+      OutEdges<I, E> outEdges = partitionEdges.get(vertexId);
+      if (outEdges == null) {
+        OutEdges<I, E> newOutEdges =
+            configuration.createAndInitializeInputOutEdges();
+        outEdges = partitionEdges.putIfAbsent(vertexId, newOutEdges);
+        if (outEdges == null) {
+          outEdges = newOutEdges;
           // Since we had to use the vertex id as a new key in the map,
           // we need to release the object.
           vertexIdEdgeIterator.releaseCurrentVertexId();
         }
       }
-      synchronized (vertexEdges) {
-        vertexEdges.add(edge);
+      synchronized (outEdges) {
+        outEdges.add(edge);
       }
     }
   }
 
   /**
-   * Convert the input edges to the {@link VertexEdges} data structure used
+   * Convert the input edges to the {@link OutEdges} data structure used
    * for computation (if different).
    *
    * @param inputEdges Input edges
    * @return Compute edges
    */
-  private VertexEdges<I, E> convertInputToComputeEdges(
-      VertexEdges<I, E> inputEdges) {
-    if (!useInputVertexEdges) {
+  private OutEdges<I, E> convertInputToComputeEdges(
+      OutEdges<I, E> inputEdges) {
+    if (!useInputOutEdges) {
       return inputEdges;
     } else {
-      VertexEdges<I, E> computeEdges =
-          configuration.createAndInitializeVertexEdges(inputEdges.size());
+      OutEdges<I, E> computeEdges =
+          configuration.createAndInitializeOutEdges(inputEdges.size());
       for (Edge<I, E> edge : inputEdges) {
         computeEdges.add(edge);
       }
@@ -186,10 +186,10 @@ public class EdgeStore<I extends WritableComparable,
             while ((partitionId = partitionIdQueue.poll()) != null) {
               Partition<I, V, E, M> partition =
                   service.getPartitionStore().getPartition(partitionId);
-              ConcurrentMap<I, VertexEdges<I, E>> partitionEdges =
+              ConcurrentMap<I, OutEdges<I, E>> partitionEdges =
                   transientEdges.remove(partitionId);
               for (I vertexId : partitionEdges.keySet()) {
-                VertexEdges<I, E> vertexEdges = convertInputToComputeEdges(
+                OutEdges<I, E> outEdges = convertInputToComputeEdges(
                     partitionEdges.remove(vertexId));
                 Vertex<I, V, E, M> vertex = partition.getVertex(vertexId);
                 // If the source vertex doesn't exist, create it. Otherwise,
@@ -197,10 +197,10 @@ public class EdgeStore<I extends WritableComparable,
                 if (vertex == null) {
                   vertex = configuration.createVertex();
                   vertex.initialize(vertexId, configuration.createVertexValue(),
-                      vertexEdges);
+                      outEdges);
                   partition.putVertex(vertex);
                 } else {
-                  vertex.setEdges(vertexEdges);
+                  vertex.setEdges(outEdges);
                   // Some Partition implementations (e.g. ByteArrayPartition)
                   // require us to put back the vertex after modifying it.
                   partition.saveVertex(vertex);

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
index 9fa7b64..fc23cf0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/HashMapEdges.java
@@ -31,7 +31,7 @@ import java.util.Iterator;
 import java.util.Map;
 
 /**
- * {@link VertexEdges} implementation backed by a {@link HashMap}.
+ * {@link OutEdges} implementation backed by a {@link HashMap}.
  * Parallel edges are not allowed.
  * Note: this implementation is optimized for fast random access and mutations,
  * but uses more space.
@@ -40,9 +40,9 @@ import java.util.Map;
  * @param <E> Edge value
  */
 public class HashMapEdges<I extends WritableComparable, E extends Writable>
-    extends ConfigurableVertexEdges<I, E>
-    implements StrictRandomAccessVertexEdges<I, E>,
-    MutableVertexEdges<I, E> {
+    extends ConfigurableOutEdges<I, E>
+    implements StrictRandomAccessOutEdges<I, E>,
+    MutableOutEdges<I, E> {
   /** Map from target vertex id to edge value. */
   private HashMap<I, E> edgeMap;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
index 123d49f..abe6151 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/HashMultimapEdges.java
@@ -31,7 +31,7 @@ import java.util.Iterator;
 import java.util.Map;
 
 /**
- * {@link VertexEdges} implementation backed by an {@link ArrayListMultimap}.
+ * {@link OutEdges} implementation backed by an {@link ArrayListMultimap}.
  * Parallel edges are allowed.
  * Note: this implementation is optimized for fast mutations,
  * but uses more space.
@@ -40,8 +40,8 @@ import java.util.Map;
  * @param <E> Edge value
  */
 public class HashMultimapEdges<I extends WritableComparable, E extends Writable>
-    extends ConfigurableVertexEdges<I, E>
-    implements MultiRandomAccessVertexEdges<I, E> {
+    extends ConfigurableOutEdges<I, E>
+    implements MultiRandomAccessOutEdges<I, E> {
   /** Multimap from target vertex id to edge values. */
   private ArrayListMultimap<I, E> edgeMultimap;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
index 2363caf..9867771 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntNullArrayEdges.java
@@ -33,14 +33,14 @@ import java.util.Collection;
 import java.util.Iterator;
 
 /**
- * Implementation of {@link VertexEdges} with int ids and null edge
+ * Implementation of {@link OutEdges} with int ids and null edge
  * values, backed by dynamic primitive array.
  * Parallel edges are allowed.
  * Note: this implementation is optimized for space usage,
  * but edge removals are expensive.
  */
 public class IntNullArrayEdges
-    implements ReuseObjectsVertexEdges<IntWritable, NullWritable> {
+    implements ReuseObjectsOutEdges<IntWritable, NullWritable> {
   /** Array of target vertex ids */
   private IntArrayList neighbors;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
index 6ce85d5..45d182e 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleArrayEdges.java
@@ -33,15 +33,15 @@ import java.util.Collection;
 import java.util.Iterator;
 
 /**
- * Implementation of {@link VertexEdges} with long ids and double edge
+ * Implementation of {@link OutEdges} with long ids and double edge
  * values, backed by dynamic primitive arrays.
  * Parallel edges are allowed.
  * Note: this implementation is optimized for space usage,
  * but edge removals are expensive.
  */
 public class LongDoubleArrayEdges
-    implements ReuseObjectsVertexEdges<LongWritable, DoubleWritable>,
-    MutableVertexEdges<LongWritable, DoubleWritable> {
+    implements ReuseObjectsOutEdges<LongWritable, DoubleWritable>,
+    MutableOutEdges<LongWritable, DoubleWritable> {
   /** Array of target vertex ids. */
   private LongArrayList neighbors;
   /** Array of edge values. */

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
index 908ad25..c33200d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongDoubleHashMapEdges.java
@@ -32,7 +32,7 @@ import java.util.Collection;
 import java.util.Iterator;
 
 /**
- * {@link VertexEdges} implementation with long ids and double edge values,
+ * {@link OutEdges} implementation with long ids and double edge values,
  * backed by a {@link Long2DoubleOpenHashMap}.
  * Parallel edges are not allowed.
  * Note: this implementation is optimized for fast random access and mutations,
@@ -40,9 +40,9 @@ import java.util.Iterator;
  * {@link LongDoubleArrayEdges}.
  */
 public class LongDoubleHashMapEdges
-    implements StrictRandomAccessVertexEdges<LongWritable, DoubleWritable>,
-    ReuseObjectsVertexEdges<LongWritable, DoubleWritable>,
-    MutableVertexEdges<LongWritable, DoubleWritable> {
+    implements StrictRandomAccessOutEdges<LongWritable, DoubleWritable>,
+    ReuseObjectsOutEdges<LongWritable, DoubleWritable>,
+    MutableOutEdges<LongWritable, DoubleWritable> {
   /** Hash map from target vertex id to edge value. */
   private Long2DoubleOpenHashMap edgeMap;
   /** Representative edge value object, used by getEdgeValue(). */

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
index fee27f1..d753588 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullArrayEdges.java
@@ -30,15 +30,15 @@ import java.util.Collection;
 import java.util.Iterator;
 
 /**
- * Implementation of {@link VertexEdges} with long ids and null edge
+ * Implementation of {@link OutEdges} with long ids and null edge
  * values, backed by a dynamic primitive array.
  * Parallel edges are allowed.
  * Note: this implementation is optimized for space usage,
  * but random access and edge removals are expensive.
  */
 public class LongNullArrayEdges
-    implements ReuseObjectsVertexEdges<LongWritable, NullWritable>,
-    MutableVertexEdges<LongWritable, NullWritable> {
+    implements ReuseObjectsOutEdges<LongWritable, NullWritable>,
+    MutableOutEdges<LongWritable, NullWritable> {
   /** Array of target vertex ids. */
   private LongArrayList neighbors;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
index f374e36..ef0302c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongNullHashSetEdges.java
@@ -30,7 +30,7 @@ import java.util.Collection;
 import java.util.Iterator;
 
 /**
- * {@link VertexEdges} implementation with long ids and null edge values,
+ * {@link OutEdges} implementation with long ids and null edge values,
  * backed by a {@link LongOpenHashSet}.
  * Parallel edges are not allowed.
  * Note: this implementation is optimized for fast random access and mutations,
@@ -38,8 +38,8 @@ import java.util.Iterator;
  * {@link LongNullArrayEdges}.
  */
 public class LongNullHashSetEdges
-    implements ReuseObjectsVertexEdges<LongWritable, NullWritable>,
-    MutableVertexEdges<LongWritable, NullWritable> {
+    implements ReuseObjectsOutEdges<LongWritable, NullWritable>,
+    MutableOutEdges<LongWritable, NullWritable> {
   /** Hash set of target vertex ids. */
   private LongOpenHashSet neighbors;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java
new file mode 100644
index 0000000..e3fc16b
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessOutEdges.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface for {@link OutEdges} implementations that provide efficient
+ * random access to the edges given the target vertex id.
+ * This version is for multigraphs (i.e. there can be parallel edges).
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface MultiRandomAccessOutEdges<I extends WritableComparable,
+    E extends Writable> extends OutEdges<I, E> {
+  /**
+   * Return an iterable over the edge values for a given target vertex id.
+   *
+   * @param targetVertexId Target vertex id
+   * @return Iterable of edge values
+   */
+  Iterable<E> getAllEdgeValues(I targetVertexId);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java
deleted file mode 100644
index 9f8658e..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MultiRandomAccessVertexEdges.java
+++ /dev/null
@@ -1,41 +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.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Interface for {@link VertexEdges} implementations that provide efficient
- * random access to the edges given the target vertex id.
- * This version is for multigraphs (i.e. there can be parallel edges).
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface MultiRandomAccessVertexEdges<I extends WritableComparable,
-    E extends Writable> extends VertexEdges<I, E> {
-  /**
-   * Return an iterable over the edge values for a given target vertex id.
-   *
-   * @param targetVertexId Target vertex id
-   * @return Iterable of edge values
-   */
-  Iterable<E> getAllEdgeValues(I targetVertexId);
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
index 9dab09c..82486f4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesIterable.java
@@ -26,7 +26,7 @@ import java.util.Iterator;
 
 /**
  * Helper class to provide a mutable iterable over the edges when the chosen
- * {@link VertexEdges} doesn't offer a specialized one.
+ * {@link OutEdges} doesn't offer a specialized one.
  *
  * @param <I> Vertex id
  * @param <E> Edge value
@@ -48,7 +48,7 @@ public class MutableEdgesIterable<I extends WritableComparable,
   @Override
   public Iterator<MutableEdge<I, E>> iterator() {
     final MutableEdgesWrapper<I, E> mutableEdgesWrapper =
-        MutableEdgesWrapper.wrap((VertexEdges<I, E>) vertex.getEdges(),
+        MutableEdgesWrapper.wrap((OutEdges<I, E>) vertex.getEdges(),
             vertex.getConf());
     vertex.setEdges(mutableEdgesWrapper);
 
@@ -57,7 +57,7 @@ public class MutableEdgesIterable<I extends WritableComparable,
       private Iterator<Edge<I, E>> oldEdgesIterator =
           mutableEdgesWrapper.getOldEdgesIterator();
       /** New edges data structure. */
-      private VertexEdges<I, E> newEdges = mutableEdgesWrapper.getNewEdges();
+      private OutEdges<I, E> newEdges = mutableEdgesWrapper.getNewEdges();
 
       @Override
       public boolean hasNext() {

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
index 5b20bf7..cd845d0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableEdgesWrapper.java
@@ -31,8 +31,8 @@ import java.util.Iterator;
  * Helper class that wraps the current out-edges and inserts them into a new
  * data structure as they are iterated over.
  * Used by Vertex to provide a mutable iterator when the chosen
- * {@link VertexEdges} doesn't offer a specialized one.
- * The edges are "unwrapped" back to the chosen {@link VertexEdges} data
+ * {@link OutEdges} doesn't offer a specialized one.
+ * The edges are "unwrapped" back to the chosen {@link OutEdges} data
  * structure as soon as possible: either when the iterator is exhausted,
  * or after compute() if iteration has been terminated early.
  *
@@ -40,9 +40,9 @@ import java.util.Iterator;
  * @param <E> Edge value
  */
 public class MutableEdgesWrapper<I extends WritableComparable,
-    E extends Writable> implements VertexEdges<I, E> {
+    E extends Writable> implements OutEdges<I, E> {
   /** New edges data structure (initially empty). */
-  private final VertexEdges<I, E> newEdges;
+  private final OutEdges<I, E> newEdges;
   /** Iterator over the old edges. */
   private final Iterator<Edge<I, E>> oldEdgesIterator;
   /** Last edge that was returned during iteration. */
@@ -55,8 +55,8 @@ public class MutableEdgesWrapper<I extends WritableComparable,
    * @param oldEdges Current out-edges
    * @param newEdges New (empty) edges data structure
    */
-  private MutableEdgesWrapper(VertexEdges<I, E> oldEdges,
-                              VertexEdges<I, E> newEdges) {
+  private MutableEdgesWrapper(OutEdges<I, E> oldEdges,
+                              OutEdges<I, E> newEdges) {
     oldEdgesIterator = oldEdges.iterator();
     this.newEdges = newEdges;
   }
@@ -72,19 +72,19 @@ public class MutableEdgesWrapper<I extends WritableComparable,
    */
   public static <I extends WritableComparable, E extends Writable>
   MutableEdgesWrapper<I, E> wrap(
-      VertexEdges<I, E> edges,
+      OutEdges<I, E> edges,
       ImmutableClassesGiraphConfiguration<I, ?, E, ?> conf) {
     MutableEdgesWrapper<I, E> wrapper = new MutableEdgesWrapper<I, E>(
-        edges, conf.createAndInitializeVertexEdges(edges.size()));
+        edges, conf.createAndInitializeOutEdges(edges.size()));
     return wrapper;
   }
 
   /**
    * Moves all the remaining edges to the new data structure, and returns it.
    *
-   * @return The new {@link VertexEdges} data structure.
+   * @return The new {@link OutEdges} data structure.
    */
-  public VertexEdges<I, E> unwrap() {
+  public OutEdges<I, E> unwrap() {
     if (currentEdge != null) {
       newEdges.add(currentEdge);
       currentEdge = null;
@@ -96,11 +96,11 @@ public class MutableEdgesWrapper<I extends WritableComparable,
   }
 
   /**
-   * Get the new {@link VertexEdges} data structure.
+   * Get the new {@link OutEdges} data structure.
    *
    * @return New edges
    */
-  public VertexEdges<I, E> getNewEdges() {
+  public OutEdges<I, E> getNewEdges() {
     return newEdges;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java
new file mode 100644
index 0000000..0dcc018
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/MutableOutEdges.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import java.util.Iterator;
+
+/**
+ * Interface for {@link OutEdges} implementations that have an optimized
+ * mutable edge iterator.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface MutableOutEdges<I extends WritableComparable,
+    E extends Writable> extends OutEdges<I, E> {
+  /**
+   * Returns an iterator over edges that can be modified in-place,
+   * either by changing the current edge value or by removing the current edge.
+   *
+   * @return A mutable edge iterator
+   */
+  Iterator<MutableEdge<I, E>> mutableIterator();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java
deleted file mode 100644
index 503de92..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/MutableVertexEdges.java
+++ /dev/null
@@ -1,42 +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.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-import java.util.Iterator;
-
-/**
- * Interface for {@link VertexEdges} implementations that have an optimized
- * mutable edge iterator.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface MutableVertexEdges<I extends WritableComparable,
-    E extends Writable> extends VertexEdges<I, E> {
-  /**
-   * Returns an iterator over edges that can be modified in-place,
-   * either by changing the current edge value or by removing the current edge.
-   *
-   * @return A mutable edge iterator
-   */
-  Iterator<MutableEdge<I, E>> mutableIterator();
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java
new file mode 100644
index 0000000..34dc5cb
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/OutEdges.java
@@ -0,0 +1,84 @@
+/*
+ * 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.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface for data structures that store out-edges for a vertex.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface OutEdges<I extends WritableComparable, E extends Writable>
+    extends Iterable<Edge<I, E>>, Writable {
+  /**
+   * Initialize the data structure and set the edges from an iterable.
+   * This method (or one of the two alternatives) must be called
+   * after instantiation, unless readFields() is called.
+   * Note: whether parallel edges are allowed or not depends on the
+   * implementation.
+   *
+   * @param edges Iterable of edges
+   */
+  void initialize(Iterable<Edge<I, E>> edges);
+
+  /**
+   * Initialize the data structure with the specified initial capacity.
+   * This method (or one of the two alternatives) must be called
+   * after instantiation, unless readFields() is called.
+   *
+   * @param capacity Initial capacity
+   */
+  void initialize(int capacity);
+
+  /**
+   * Initialize the data structure with the default initial capacity.
+   * This method (or one of the two alternatives) must be called
+   * after instantiation, unless readFields() is called.
+   *
+   */
+  void initialize();
+
+  /**
+   * Add an edge.
+   * Note: whether parallel edges are allowed or not depends on the
+   * implementation.
+   *
+   * @param edge Edge to add
+   */
+  void add(Edge<I, E> edge);
+
+  /**
+   * Remove all edges to the given target vertex.
+   * Note: the implementation will vary depending on whether parallel edges
+   * are allowed or not.
+   *
+   * @param targetVertexId Target vertex id
+   */
+  void remove(I targetVertexId);
+
+  /**
+   * Return the number of edges.
+   *
+   * @return Number of edges
+   */
+  int size();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java
new file mode 100644
index 0000000..27cf9ca
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsOutEdges.java
@@ -0,0 +1,34 @@
+/*
+ * 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.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Empty interface to characterize {@link OutEdges} implementations that
+ * don't keep references to the Edge (or id and value) objects they are passed.
+ * The Giraph infrastructure can exploit this characteristic by reusing Edge
+ * objects.
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface ReuseObjectsOutEdges<I extends WritableComparable,
+    E extends Writable> extends OutEdges<I, E> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java
deleted file mode 100644
index 7704baf..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/ReuseObjectsVertexEdges.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Empty interface to characterize {@link VertexEdges} implementations that
- * don't keep references to the Edge (or id and value) objects they are passed.
- * The Giraph infrastructure can exploit this characteristic by reusing Edge
- * objects.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface ReuseObjectsVertexEdges<I extends WritableComparable,
-    E extends Writable> extends VertexEdges<I, E> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java
new file mode 100644
index 0000000..800f45b
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessOutEdges.java
@@ -0,0 +1,51 @@
+/*
+ * 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.edge;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface for {@link OutEdges} implementations that provide efficient
+ * random access to the edges given the target vertex id.
+ * This version is for strict graphs (i.e. assumes no parallel edges).
+ *
+ * @param <I> Vertex id
+ * @param <E> Edge value
+ */
+public interface StrictRandomAccessOutEdges<I extends WritableComparable,
+    E extends Writable> extends OutEdges<I, E> {
+  /**
+   * Return the edge value for the given target vertex id (or null if there
+   * is no edge pointing to it).
+   *
+   * @param targetVertexId Target vertex id
+   * @return Edge value
+   */
+  E getEdgeValue(I targetVertexId);
+
+  /**
+   * Set the edge value for the given target vertex id (if an edge to that
+   * vertex exists).
+   *
+   * @param targetVertexId Target vertex id
+   * @param edgeValue Edge value
+   */
+  void setEdgeValue(I targetVertexId, E edgeValue);
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java
deleted file mode 100644
index 5fdc2d2..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/StrictRandomAccessVertexEdges.java
+++ /dev/null
@@ -1,51 +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.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Interface for {@link VertexEdges} implementations that provide efficient
- * random access to the edges given the target vertex id.
- * This version is for strict graphs (i.e. assumes no parallel edges).
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface StrictRandomAccessVertexEdges<I extends WritableComparable,
-    E extends Writable> extends VertexEdges<I, E> {
-  /**
-   * Return the edge value for the given target vertex id (or null if there
-   * is no edge pointing to it).
-   *
-   * @param targetVertexId Target vertex id
-   * @return Edge value
-   */
-  E getEdgeValue(I targetVertexId);
-
-  /**
-   * Set the edge value for the given target vertex id (if an edge to that
-   * vertex exists).
-   *
-   * @param targetVertexId Target vertex id
-   * @param edgeValue Edge value
-   */
-  void setEdgeValue(I targetVertexId, E edgeValue);
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java
deleted file mode 100644
index bb885b7..0000000
--- a/giraph-core/src/main/java/org/apache/giraph/edge/VertexEdges.java
+++ /dev/null
@@ -1,84 +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.edge;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
-/**
- * Interface for data structures that store out-edges for a vertex.
- *
- * @param <I> Vertex id
- * @param <E> Edge value
- */
-public interface VertexEdges<I extends WritableComparable, E extends Writable>
-    extends Iterable<Edge<I, E>>, Writable {
-  /**
-   * Initialize the data structure and set the edges from an iterable.
-   * This method (or one of the two alternatives) must be called
-   * after instantiation, unless readFields() is called.
-   * Note: whether parallel edges are allowed or not depends on the
-   * implementation.
-   *
-   * @param edges Iterable of edges
-   */
-  void initialize(Iterable<Edge<I, E>> edges);
-
-  /**
-   * Initialize the data structure with the specified initial capacity.
-   * This method (or one of the two alternatives) must be called
-   * after instantiation, unless readFields() is called.
-   *
-   * @param capacity Initial capacity
-   */
-  void initialize(int capacity);
-
-  /**
-   * Initialize the data structure with the default initial capacity.
-   * This method (or one of the two alternatives) must be called
-   * after instantiation, unless readFields() is called.
-   *
-   */
-  void initialize();
-
-  /**
-   * Add an edge.
-   * Note: whether parallel edges are allowed or not depends on the
-   * implementation.
-   *
-   * @param edge Edge to add
-   */
-  void add(Edge<I, E> edge);
-
-  /**
-   * Remove all edges to the given target vertex.
-   * Note: the implementation will vary depending on whether parallel edges
-   * are allowed or not.
-   *
-   * @param targetVertexId Target vertex id
-   */
-  void remove(I targetVertexId);
-
-  /**
-   * Return the number of edges.
-   *
-   * @return Number of edges
-   */
-  int size();
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java b/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
index ade25fe..61624e5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/Vertex.java
@@ -21,13 +21,13 @@ package org.apache.giraph.graph;
 import com.google.common.collect.UnmodifiableIterator;
 import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.edge.Edge;
-import org.apache.giraph.edge.MultiRandomAccessVertexEdges;
+import org.apache.giraph.edge.MultiRandomAccessOutEdges;
 import org.apache.giraph.edge.MutableEdge;
 import org.apache.giraph.edge.MutableEdgesIterable;
 import org.apache.giraph.edge.MutableEdgesWrapper;
-import org.apache.giraph.edge.MutableVertexEdges;
-import org.apache.giraph.edge.StrictRandomAccessVertexEdges;
-import org.apache.giraph.edge.VertexEdges;
+import org.apache.giraph.edge.MutableOutEdges;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.edge.StrictRandomAccessOutEdges;
 import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.worker.WorkerAggregatorUsage;
 import org.apache.giraph.worker.WorkerContext;
@@ -57,7 +57,7 @@ public abstract class Vertex<I extends WritableComparable,
   /** Vertex value. */
   private V value;
   /** Outgoing edges. */
-  private VertexEdges<I, E> edges;
+  private OutEdges<I, E> edges;
   /** If true, do not do anymore computation on this vertex. */
   private boolean halt;
   /** Global graph state **/
@@ -89,7 +89,7 @@ public abstract class Vertex<I extends WritableComparable,
   public void initialize(I id, V value) {
     this.id = id;
     this.value = value;
-    this.edges = getConf().createAndInitializeVertexEdges(0);
+    this.edges = getConf().createAndInitializeOutEdges(0);
   }
 
   /**
@@ -98,13 +98,13 @@ public abstract class Vertex<I extends WritableComparable,
    * @param edges Iterable of edges
    */
   public void setEdges(Iterable<Edge<I, E>> edges) {
-    // If the iterable is actually an instance of VertexEdges,
+    // If the iterable is actually an instance of OutEdges,
     // we simply take the reference.
-    // Otherwise, we initialize a new VertexEdges.
-    if (edges instanceof VertexEdges) {
-      this.edges = (VertexEdges<I, E>) edges;
+    // Otherwise, we initialize a new OutEdges.
+    if (edges instanceof OutEdges) {
+      this.edges = (OutEdges<I, E>) edges;
     } else {
-      this.edges = getConf().createAndInitializeVertexEdges(edges);
+      this.edges = getConf().createAndInitializeOutEdges(edges);
     }
   }
 
@@ -195,14 +195,14 @@ public abstract class Vertex<I extends WritableComparable,
    * @return An iterable of mutable out-edges
    */
   public Iterable<MutableEdge<I, E>> getMutableEdges() {
-    // If the VertexEdges implementation has a specialized mutable iterator,
+    // If the OutEdges implementation has a specialized mutable iterator,
     // we use that; otherwise, we build a new data structure as we iterate
     // over the current edges.
-    if (edges instanceof MutableVertexEdges) {
+    if (edges instanceof MutableOutEdges) {
       return new Iterable<MutableEdge<I, E>>() {
         @Override
         public Iterator<MutableEdge<I, E>> iterator() {
-          return ((MutableVertexEdges<I, E>) edges).mutableIterator();
+          return ((MutableOutEdges<I, E>) edges).mutableIterator();
         }
       };
     } else {
@@ -212,8 +212,9 @@ public abstract class Vertex<I extends WritableComparable,
 
   /**
    * If a {@link MutableEdgesWrapper} was used to provide a mutable iterator,
-   * copy any remaining edges to the new {@link VertexEdges} data
-   * structure and keep a direct reference to it (thus discarding the wrapper).
+   * copy any remaining edges to the new {@link org.apache.giraph.edge.OutEdges}
+   * data structure and keep a direct reference to it (thus discarding the
+   * wrapper).
    * Called by the Giraph infrastructure after computation.
    */
   public void unwrapMutableEdges() {
@@ -242,10 +243,10 @@ public abstract class Vertex<I extends WritableComparable,
    * @return Edge value (or null if missing)
    */
   public E getEdgeValue(I targetVertexId) {
-    // If the VertexEdges implementation has a specialized random-access
+    // If the OutEdges implementation has a specialized random-access
     // method, we use that; otherwise, we scan the edges.
-    if (edges instanceof StrictRandomAccessVertexEdges) {
-      return ((StrictRandomAccessVertexEdges<I, E>) edges)
+    if (edges instanceof StrictRandomAccessOutEdges) {
+      return ((StrictRandomAccessOutEdges<I, E>) edges)
           .getEdgeValue(targetVertexId);
     } else {
       for (Edge<I, E> edge : edges) {
@@ -265,10 +266,10 @@ public abstract class Vertex<I extends WritableComparable,
    * @param edgeValue Edge value
    */
   public void setEdgeValue(I targetVertexId, E edgeValue) {
-    // If the VertexEdges implementation has a specialized random-access
+    // If the OutEdges implementation has a specialized random-access
     // method, we use that; otherwise, we scan the edges.
-    if (edges instanceof StrictRandomAccessVertexEdges) {
-      ((StrictRandomAccessVertexEdges<I, E>) edges).setEdgeValue(
+    if (edges instanceof StrictRandomAccessOutEdges) {
+      ((StrictRandomAccessOutEdges<I, E>) edges).setEdgeValue(
           targetVertexId, edgeValue);
     } else {
       for (MutableEdge<I, E> edge : getMutableEdges()) {
@@ -291,10 +292,10 @@ public abstract class Vertex<I extends WritableComparable,
    * @return Iterable of edge values
    */
   public Iterable<E> getAllEdgeValues(final I targetVertexId) {
-    // If the VertexEdges implementation has a specialized random-access
+    // If the OutEdges implementation has a specialized random-access
     // method, we use that; otherwise, we scan the edges.
-    if (edges instanceof MultiRandomAccessVertexEdges) {
-      return ((MultiRandomAccessVertexEdges<I, E>) edges)
+    if (edges instanceof MultiRandomAccessOutEdges) {
+      return ((MultiRandomAccessOutEdges<I, E>) edges)
           .getAllEdgeValues(targetVertexId);
     } else {
       return new Iterable<E>() {
@@ -405,7 +406,7 @@ public abstract class Vertex<I extends WritableComparable,
    * @param value Vertex value
    * @param edges Initial edges
    */
-  public void addVertexRequest(I id, V value, VertexEdges<I, E> edges)
+  public void addVertexRequest(I id, V value, OutEdges<I, E> edges)
     throws IOException {
     Vertex<I, V, E, M> vertex = getConf().createVertex();
     vertex.initialize(id, value, edges);
@@ -420,7 +421,7 @@ public abstract class Vertex<I extends WritableComparable,
    * @param value Vertex value
    */
   public void addVertexRequest(I id, V value) throws IOException {
-    addVertexRequest(id, value, getConf().createAndInitializeVertexEdges());
+    addVertexRequest(id, value, getConf().createAndInitializeOutEdges());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/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 2cc4dba..48544b0 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
@@ -72,7 +72,7 @@ public class PseudoRandomEdgeInputFormat
     /** Current vertex id. */
     private LongWritable currentVertexId = new LongWritable(-1);
     /** Edges read for the current vertex. */
-    private int currentVertexEdgesRead = 0;
+    private int currentOutEdgesRead = 0;
     /** Target vertices of edges for current vertex. */
     private Set<LongWritable> currentVertexDestVertices = Sets.newHashSet();
     /** Random number generator for the current vertex (for consistency
@@ -131,20 +131,20 @@ public class PseudoRandomEdgeInputFormat
     public boolean nextEdge() throws IOException, InterruptedException {
       return totalSplitVertices > verticesRead + 1 ||
           (totalSplitVertices == verticesRead + 1 &&
-              edgesPerVertex > currentVertexEdgesRead);
+              edgesPerVertex > currentOutEdgesRead);
     }
 
     @Override
     public LongWritable getCurrentSourceId() throws IOException,
         InterruptedException {
-      if (currentVertexEdgesRead == edgesPerVertex) {
+      if (currentOutEdgesRead == edgesPerVertex) {
         ++verticesRead;
         currentVertexId = new LongWritable(-1);
       }
 
       if (currentVertexId.get() == -1) {
         currentVertexId.set(startingVertexId + verticesRead);
-        currentVertexEdgesRead = 0;
+        currentOutEdgesRead = 0;
         // Seed on the vertex id to keep the vertex data the same when
         // on different number of workers, but other parameters are the
         // same.
@@ -162,7 +162,7 @@ public class PseudoRandomEdgeInputFormat
         destVertexId.set(localEdgesHelper.generateDestVertex(
             currentVertexId.get(), random));
       } while (currentVertexDestVertices.contains(destVertexId));
-      ++currentVertexEdgesRead;
+      ++currentOutEdgesRead;
       currentVertexDestVertices.add(destVertexId);
       if (LOG.isTraceEnabled()) {
         LOG.trace("getCurrentEdge: Return edge (" + currentVertexId + ", " +
@@ -178,7 +178,7 @@ public class PseudoRandomEdgeInputFormat
 
     @Override
     public float getProgress() throws IOException, InterruptedException {
-      return (verticesRead * edgesPerVertex + currentVertexEdgesRead) *
+      return (verticesRead * edgesPerVertex + currentOutEdgesRead) *
           100.0f / (totalSplitVertices * edgesPerVertex);
     }
   }

http://git-wip-us.apache.org/repos/asf/giraph/blob/158c0b11/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
index b27fcc8..c9390ba 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/PseudoRandomIntNullVertexInputFormat.java
@@ -20,8 +20,8 @@ package org.apache.giraph.io.formats;
 
 import org.apache.giraph.bsp.BspInputSplit;
 import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.edge.ReusableEdge;
-import org.apache.giraph.edge.VertexEdges;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexReader;
@@ -121,8 +121,8 @@ public class PseudoRandomIntNullVertexInputFormat extends
       Vertex<IntWritable, FloatWritable, NullWritable, ?> vertex =
           getConf().createVertex();
       int vertexId = startingVertexId + verticesRead;
-      VertexEdges<IntWritable, NullWritable> edges =
-          getConf().createVertexEdges();
+      OutEdges<IntWritable, NullWritable> edges =
+          getConf().createOutEdges();
       edges.initialize(edgesPerVertex);
       destVertices.clear();
       for (int i = 0; i < edgesPerVertex; ++i) {