You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ni...@apache.org on 2013/06/10 22:42:35 UTC

git commit: updated refs/heads/trunk to 90f3116

Updated Branches:
  refs/heads/trunk 85dae7f85 -> 90f3116e3


GIRAPH-468: Make Vertex an Interface (nitay)


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

Branch: refs/heads/trunk
Commit: 90f3116e38f7ba54b654345ef2774aa2a99412e8
Parents: 85dae7f
Author: Nitay Joffe <ni...@apache.org>
Authored: Mon Jun 10 16:40:44 2013 -0400
Committer: Nitay Joffe <ni...@apache.org>
Committed: Mon Jun 10 16:41:33 2013 -0400

----------------------------------------------------------------------
 CHANGELOG                                       |   2 +
 .../edgemarker/AccumuloEdgeInputFormat.java     |  10 +-
 .../apache/giraph/comm/SendMutationsCache.java  |   2 +-
 .../apache/giraph/comm/SendPartitionCache.java  |  11 +-
 .../comm/WorkerClientRequestProcessor.java      |   2 +-
 .../NettyWorkerClientRequestProcessor.java      |   2 +-
 .../ImmutableClassesGiraphConfiguration.java    |   7 +-
 .../java/org/apache/giraph/edge/EdgeStore.java  |  12 +-
 .../org/apache/giraph/graph/DefaultVertex.java  | 232 +++++++++++++++++
 .../java/org/apache/giraph/graph/Vertex.java    | 252 +++++--------------
 .../apache/giraph/graph/VertexMutations.java    |   3 +-
 .../java/org/apache/giraph/io/VertexReader.java |   3 +-
 .../org/apache/giraph/io/VertexValueReader.java |   3 +-
 ...nLongDoubleFloatDoubleVertexInputFormat.java |   3 +-
 .../formats/PseudoRandomVertexInputFormat.java  |  12 +-
 .../formats/SequenceFileVertexInputFormat.java  |   5 +-
 .../io/formats/TextVertexInputFormat.java       |   5 +-
 .../io/iterables/VertexReaderWrapper.java       |   4 +-
 .../partition/DiskBackedPartitionStore.java     |   4 +-
 .../org/apache/giraph/partition/Partition.java  |   1 -
 .../giraph/utils/InMemoryVertexInputFormat.java |   9 +-
 .../java/org/apache/giraph/utils/TestGraph.java |   1 -
 .../apache/giraph/graph/TestVertexAndEdges.java |   2 +-
 ...TestAdjacencyListTextVertexOutputFormat.java |   4 +-
 ...bleDoubleAdjacencyListVertexInputFormat.java |   4 +-
 ...bleDoubleAdjacencyListVertexInputFormat.java |   6 +-
 .../partition/TestGiraphTransferRegulator.java  |   5 +-
 .../giraph/partition/TestPartitionStores.java   |   3 +-
 .../java/org/apache/giraph/utils/MockUtils.java |   4 +-
 .../LongDoubleDoubleTextInputFormat.java        |   5 +-
 .../examples/LongDoubleNullTextInputFormat.java |   3 +-
 ...malizingLongDoubleDoubleTextInputFormat.java |  10 +-
 .../examples/SimplePageRankComputation.java     |  14 +-
 .../examples/SimpleSuperstepComputation.java    |  11 +-
 .../java/org/apache/giraph/TestBspBasic.java    |   1 -
 ...nectedComponentsComputationTestInMemory.java |  10 +-
 .../SimpleShortestPathsComputationTest.java     |   5 +-
 .../SimpleTriangleClosingComputationTest.java   |  12 +-
 .../hbase/edgemarker/TableEdgeInputFormat.java  |   7 +-
 .../hbase/edgemarker/TableEdgeOutputFormat.java |   2 +-
 .../io/hcatalog/HCatalogVertexInputFormat.java  |   8 +-
 41 files changed, 409 insertions(+), 292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index a020846..e66ec42 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 1.1.0 - unreleased
+  GIRAPH-468: Make Vertex an Interface (nitay)
+
   GIRAPH-667: all workers suspended at 'saveVertices' when use 
   out of core (claudio)
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
index 108ae61..fec4783 100644
--- a/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
+++ b/giraph-accumulo/src/test/java/org/apache/giraph/io/accumulo/edgemarker/AccumuloEdgeInputFormat.java
@@ -17,10 +17,6 @@
  */
 package org.apache.giraph.io.accumulo.edgemarker;
 
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.List;
-import java.util.regex.Pattern;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.giraph.edge.Edge;
@@ -33,6 +29,12 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Pattern;
+
 /*
  Example subclass which reads in Key/Value pairs to construct vertex objects.
  */

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
index 9348e61..a1f08c0 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendMutationsCache.java
@@ -18,8 +18,8 @@
 package org.apache.giraph.comm;
 
 import org.apache.giraph.edge.Edge;
-import org.apache.giraph.graph.VertexMutations;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.graph.VertexMutations;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java b/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
index 68ce095..524c9f1 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
@@ -17,8 +17,6 @@
  */
 package org.apache.giraph.comm;
 
-import com.google.common.collect.Maps;
-import java.util.Map;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.GiraphTransferRegulator;
 import org.apache.giraph.graph.Vertex;
@@ -29,6 +27,10 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Maps;
+
+import java.util.Map;
+
 /**
  * Caches partition vertices prior to sending.  Aggregating these requests
  * will make larger, more efficient requests.  Not thread-safe.
@@ -86,7 +88,7 @@ public class SendPartitionCache<I extends WritableComparable,
    * @return A partition to send or null, if requirements are not met
    */
   public Partition<I, V, E> addVertex(PartitionOwner partitionOwner,
-                                         Vertex<I, V, E> vertex) {
+      Vertex<I, V, E> vertex) {
     Partition<I, V, E> partition =
         ownerPartitionMap.get(partitionOwner);
     if (partition == null) {
@@ -97,8 +99,7 @@ public class SendPartitionCache<I extends WritableComparable,
     }
     transferRegulator.incrementCounters(partitionOwner, vertex);
 
-    Vertex<I, V, E> oldVertex =
-        partition.putVertex(vertex);
+    Vertex<I, V, E> oldVertex = partition.putVertex(vertex);
     if (oldVertex != null) {
       LOG.warn("addVertex: Replacing vertex " + oldVertex +
           " with " + vertex);

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
index 731d0ee..89fb3e4 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/WorkerClientRequestProcessor.java
@@ -18,9 +18,9 @@
 package org.apache.giraph.comm;
 
 import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionOwner;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.worker.WorkerInfo;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
index 06965ca..2adf19d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
+++ b/giraph-core/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
@@ -257,7 +257,7 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
 
   @Override
   public void sendVertexRequest(PartitionOwner partitionOwner,
-                                Vertex<I, V, E> vertex) {
+      Vertex<I, V, E> vertex) {
     Partition<I, V, E> partition =
         sendPartitionCache.addVertex(partitionOwner, vertex);
     if (partition == null) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/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 aa52498..dbd2452 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
@@ -20,11 +20,12 @@ package org.apache.giraph.conf;
 
 import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
-import org.apache.giraph.graph.Computation;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.edge.OutEdges;
 import org.apache.giraph.edge.ReusableEdge;
+import org.apache.giraph.graph.DefaultVertex;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
 import org.apache.giraph.graph.VertexValueFactory;
@@ -464,7 +465,9 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    * @return Instantiated vertex
    */
   public Vertex<I, V, E> createVertex() {
-    return ReflectionUtils.newInstance(Vertex.class, this);
+    Vertex<I, V, E> vertex = new DefaultVertex<I, V, E>();
+    vertex.setConf(this);
+    return vertex;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/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 420bf93..23df689 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
@@ -18,11 +18,6 @@
 
 package org.apache.giraph.edge;
 
-import com.google.common.collect.MapMaker;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentMap;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.Vertex;
@@ -35,6 +30,13 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.MapMaker;
+
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentMap;
+
 /**
  * Collects incoming edges for vertices owned by this worker.
  *

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertex.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertex.java b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertex.java
new file mode 100644
index 0000000..03cb3c1
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertex.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.giraph.graph;
+
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.edge.Edge;
+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.MutableOutEdges;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.edge.StrictRandomAccessOutEdges;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.collect.UnmodifiableIterator;
+
+import java.util.Iterator;
+
+/**
+ * Class which holds vertex id, data and edges.
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ */
+public class DefaultVertex<I extends WritableComparable,
+    V extends Writable, E extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E>
+    implements Vertex<I, V, E> {
+  /** Vertex id. */
+  private I id;
+  /** Vertex value. */
+  private V value;
+  /** Outgoing edges. */
+  private OutEdges<I, E> edges;
+  /** If true, do not do anymore computation on this vertex. */
+  private boolean halt;
+
+  @Override
+  public void initialize(I id, V value, Iterable<Edge<I, E>> edges) {
+    this.id = id;
+    this.value = value;
+    setEdges(edges);
+  }
+
+  @Override
+  public void initialize(I id, V value) {
+    this.id = id;
+    this.value = value;
+    this.edges = getConf().createAndInitializeOutEdges(0);
+  }
+
+  @Override
+  public void setEdges(Iterable<Edge<I, E>> edges) {
+    // If the iterable is actually an instance of OutEdges,
+    // we simply take the reference.
+    // Otherwise, we initialize a new OutEdges.
+    if (edges instanceof OutEdges) {
+      this.edges = (OutEdges<I, E>) edges;
+    } else {
+      this.edges = getConf().createAndInitializeOutEdges(edges);
+    }
+  }
+
+  @Override
+  public I getId() {
+    return id;
+  }
+
+  @Override
+  public V getValue() {
+    return value;
+  }
+
+  @Override
+  public void setValue(V value) {
+    this.value = value;
+  }
+
+  @Override
+  public Iterable<Edge<I, E>> getEdges() {
+    return edges;
+  }
+
+  @Override
+  public Iterable<MutableEdge<I, E>> getMutableEdges() {
+    // 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 MutableOutEdges) {
+      return new Iterable<MutableEdge<I, E>>() {
+        @Override
+        public Iterator<MutableEdge<I, E>> iterator() {
+          return ((MutableOutEdges<I, E>) edges).mutableIterator();
+        }
+      };
+    } else {
+      return new MutableEdgesIterable<I, E>(this);
+    }
+  }
+
+  @Override
+  public void unwrapMutableEdges() {
+    if (edges instanceof MutableEdgesWrapper) {
+      edges = ((MutableEdgesWrapper<I, E>) edges).unwrap();
+    }
+  }
+
+  @Override
+  public int getNumEdges() {
+    return edges.size();
+  }
+
+  @Override
+  public E getEdgeValue(I targetVertexId) {
+    // If the OutEdges implementation has a specialized random-access
+    // method, we use that; otherwise, we scan the edges.
+    if (edges instanceof StrictRandomAccessOutEdges) {
+      return ((StrictRandomAccessOutEdges<I, E>) edges)
+          .getEdgeValue(targetVertexId);
+    } else {
+      for (Edge<I, E> edge : edges) {
+        if (edge.getTargetVertexId().equals(targetVertexId)) {
+          return edge.getValue();
+        }
+      }
+      return null;
+    }
+  }
+
+  @Override
+  public void setEdgeValue(I targetVertexId, E edgeValue) {
+    // If the OutEdges implementation has a specialized random-access
+    // method, we use that; otherwise, we scan the edges.
+    if (edges instanceof StrictRandomAccessOutEdges) {
+      ((StrictRandomAccessOutEdges<I, E>) edges).setEdgeValue(
+          targetVertexId, edgeValue);
+    } else {
+      for (MutableEdge<I, E> edge : getMutableEdges()) {
+        if (edge.getTargetVertexId().equals(targetVertexId)) {
+          edge.setValue(edgeValue);
+        }
+      }
+    }
+  }
+
+  @Override
+  public Iterable<E> getAllEdgeValues(final I targetVertexId) {
+    // If the OutEdges implementation has a specialized random-access
+    // method, we use that; otherwise, we scan the edges.
+    if (edges instanceof MultiRandomAccessOutEdges) {
+      return ((MultiRandomAccessOutEdges<I, E>) edges)
+          .getAllEdgeValues(targetVertexId);
+    } else {
+      return new Iterable<E>() {
+        @Override
+        public Iterator<E> iterator() {
+          return new UnmodifiableIterator<E>() {
+            /** Iterator over all edges. */
+            private Iterator<Edge<I, E>> edgeIterator = edges.iterator();
+            /** Last matching edge found. */
+            private Edge<I, E> currentEdge;
+
+            @Override
+            public boolean hasNext() {
+              while (edgeIterator.hasNext()) {
+                currentEdge = edgeIterator.next();
+                if (currentEdge.getTargetVertexId().equals(targetVertexId)) {
+                  return true;
+                }
+              }
+              return false;
+            }
+
+            @Override
+            public E next() {
+              return currentEdge.getValue();
+            }
+          };
+        }
+      };
+    }
+  }
+
+  @Override
+  public void voteToHalt() {
+    halt = true;
+  }
+
+  @Override
+  public void wakeUp() {
+    halt = false;
+  }
+
+  @Override
+  public boolean isHalted() {
+    return halt;
+  }
+
+  @Override
+  public void addEdge(Edge<I, E> edge) {
+    edges.add(edge);
+  }
+
+  @Override
+  public void removeEdges(I targetVertexId) {
+    edges.remove(targetVertexId);
+  }
+
+  @Override
+  public String toString() {
+    return "Vertex(id=" + getId() + ",value=" + getValue() +
+        ",#edges=" + getNumEdges() + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/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 1241ae7..5c6ca1d 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
@@ -18,21 +18,12 @@
 
 package org.apache.giraph.graph;
 
-import com.google.common.collect.UnmodifiableIterator;
-import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.edge.Edge;
-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.MutableOutEdges;
-import org.apache.giraph.edge.OutEdges;
-import org.apache.giraph.edge.StrictRandomAccessOutEdges;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
-import java.util.Iterator;
-
 /**
  * Class which holds vertex id, data and edges.
  *
@@ -40,18 +31,9 @@ import java.util.Iterator;
  * @param <V> Vertex data
  * @param <E> Edge data
  */
-public class Vertex<I extends WritableComparable,
-    V extends Writable, E extends Writable>
-    extends DefaultImmutableClassesGiraphConfigurable<I, V, E> {
-  /** Vertex id. */
-  private I id;
-  /** Vertex value. */
-  private V value;
-  /** Outgoing edges. */
-  private OutEdges<I, E> edges;
-  /** If true, do not do anymore computation on this vertex. */
-  private boolean halt;
-
+public interface Vertex<I extends WritableComparable,
+    V extends Writable, E extends Writable> extends
+    ImmutableClassesGiraphConfigurable<I, V, E> {
   /**
    * Initialize id, value, and edges.
    * This method (or the alternative form initialize(id, value)) must be called
@@ -61,11 +43,7 @@ public class Vertex<I extends WritableComparable,
    * @param value Vertex value
    * @param edges Iterable of edges
    */
-  public void initialize(I id, V value, Iterable<Edge<I, E>> edges) {
-    this.id = id;
-    this.value = value;
-    setEdges(edges);
-  }
+  void initialize(I id, V value, Iterable<Edge<I, E>> edges);
 
   /**
    * Initialize id and value. Vertex edges will be empty.
@@ -75,54 +53,43 @@ public class Vertex<I extends WritableComparable,
    * @param id Vertex id
    * @param value Vertex value
    */
-  public void initialize(I id, V value) {
-    this.id = id;
-    this.value = value;
-    this.edges = getConf().createAndInitializeOutEdges(0);
-  }
-
-  /**
-   * Set the outgoing edges for this vertex.
-   *
-   * @param edges Iterable of edges
-   */
-  public void setEdges(Iterable<Edge<I, E>> edges) {
-    // If the iterable is actually an instance of OutEdges,
-    // we simply take the reference.
-    // Otherwise, we initialize a new OutEdges.
-    if (edges instanceof OutEdges) {
-      this.edges = (OutEdges<I, E>) edges;
-    } else {
-      this.edges = getConf().createAndInitializeOutEdges(edges);
-    }
-  }
+  void initialize(I id, V value);
 
   /**
    * Get the vertex id.
    *
    * @return My vertex id.
    */
-  public I getId() {
-    return id;
-  }
+  I getId();
 
   /**
    * Get the vertex value (data stored with vertex)
    *
    * @return Vertex value
    */
-  public V getValue() {
-    return value;
-  }
+  V getValue();
 
   /**
    * Set the vertex data (immediately visible in the computation)
    *
    * @param value Vertex data to be set
    */
-  public void setValue(V value) {
-    this.value = value;
-  }
+  void setValue(V value);
+
+  /**
+   * After this is called, the compute() code will no longer be called for
+   * this vertex unless a message is sent to it.  Then the compute() code
+   * will be called once again until this function is called.  The
+   * application finishes only when all vertices vote to halt.
+   */
+  void voteToHalt();
+
+  /**
+   * Get the number of outgoing edges on this vertex.
+   *
+   * @return the total number of outbound edges from this vertex
+   */
+  int getNumEdges();
 
   /**
    * Get a read-only view of the out-edges of this vertex.
@@ -134,9 +101,14 @@ public class Vertex<I extends WritableComparable,
    *
    * @return the out edges (sort order determined by subclass implementation).
    */
-  public Iterable<Edge<I, E>> getEdges() {
-    return edges;
-  }
+  Iterable<Edge<I, E>> getEdges();
+
+  /**
+   * Set the outgoing edges for this vertex.
+   *
+   * @param edges Iterable of edges
+   */
+  void setEdges(Iterable<Edge<I, E>> edges);
 
   /**
    * Get an iterable of out-edges that can be modified in-place.
@@ -147,43 +119,7 @@ public class Vertex<I extends WritableComparable,
    *
    * @return An iterable of mutable out-edges
    */
-  public Iterable<MutableEdge<I, E>> getMutableEdges() {
-    // 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 MutableOutEdges) {
-      return new Iterable<MutableEdge<I, E>>() {
-        @Override
-        public Iterator<MutableEdge<I, E>> iterator() {
-          return ((MutableOutEdges<I, E>) edges).mutableIterator();
-        }
-      };
-    } else {
-      return new MutableEdgesIterable<I, E>(this);
-    }
-  }
-
-  /**
-   * If a {@link MutableEdgesWrapper} was used to provide a mutable iterator,
-   * 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() {
-    if (edges instanceof MutableEdgesWrapper) {
-      edges = ((MutableEdgesWrapper<I, E>) edges).unwrap();
-    }
-  }
-
-  /**
-   * Get the number of outgoing edges on this vertex.
-   *
-   * @return the total number of outbound edges from this vertex
-   */
-  public int getNumEdges() {
-    return edges.size();
-  }
+  Iterable<MutableEdge<I, E>> getMutableEdges();
 
   /**
    * Return the value of the first edge with the given target vertex id,
@@ -195,21 +131,7 @@ public class Vertex<I extends WritableComparable,
    * @param targetVertexId Target vertex id
    * @return Edge value (or null if missing)
    */
-  public E getEdgeValue(I targetVertexId) {
-    // If the OutEdges implementation has a specialized random-access
-    // method, we use that; otherwise, we scan the edges.
-    if (edges instanceof StrictRandomAccessOutEdges) {
-      return ((StrictRandomAccessOutEdges<I, E>) edges)
-          .getEdgeValue(targetVertexId);
-    } else {
-      for (Edge<I, E> edge : edges) {
-        if (edge.getTargetVertexId().equals(targetVertexId)) {
-          return edge.getValue();
-        }
-      }
-      return null;
-    }
-  }
+  E getEdgeValue(I targetVertexId);
 
   /**
    * If an edge to the target vertex exists, set it to the given edge value.
@@ -218,20 +140,7 @@ public class Vertex<I extends WritableComparable,
    * @param targetVertexId Target vertex id
    * @param edgeValue Edge value
    */
-  public void setEdgeValue(I targetVertexId, E edgeValue) {
-    // If the OutEdges implementation has a specialized random-access
-    // method, we use that; otherwise, we scan the edges.
-    if (edges instanceof StrictRandomAccessOutEdges) {
-      ((StrictRandomAccessOutEdges<I, E>) edges).setEdgeValue(
-          targetVertexId, edgeValue);
-    } else {
-      for (MutableEdge<I, E> edge : getMutableEdges()) {
-        if (edge.getTargetVertexId().equals(targetVertexId)) {
-          edge.setValue(edgeValue);
-        }
-      }
-    }
-  }
+  void setEdgeValue(I targetVertexId, E edgeValue);
 
   /**
    * Get an iterable over the values of all edges with the given target
@@ -244,90 +153,41 @@ public class Vertex<I extends WritableComparable,
    * @param targetVertexId Target vertex id
    * @return Iterable of edge values
    */
-  public Iterable<E> getAllEdgeValues(final I targetVertexId) {
-    // If the OutEdges implementation has a specialized random-access
-    // method, we use that; otherwise, we scan the edges.
-    if (edges instanceof MultiRandomAccessOutEdges) {
-      return ((MultiRandomAccessOutEdges<I, E>) edges)
-          .getAllEdgeValues(targetVertexId);
-    } else {
-      return new Iterable<E>() {
-        @Override
-        public Iterator<E> iterator() {
-          return new UnmodifiableIterator<E>() {
-            /** Iterator over all edges. */
-            private Iterator<Edge<I, E>> edgeIterator = edges.iterator();
-            /** Last matching edge found. */
-            private Edge<I, E> currentEdge;
-
-            @Override
-            public boolean hasNext() {
-              while (edgeIterator.hasNext()) {
-                currentEdge = edgeIterator.next();
-                if (currentEdge.getTargetVertexId().equals(targetVertexId)) {
-                  return true;
-                }
-              }
-              return false;
-            }
-
-            @Override
-            public E next() {
-              return currentEdge.getValue();
-            }
-          };
-        }
-      };
-    }
-  }
+  Iterable<E> getAllEdgeValues(final I targetVertexId);
 
   /**
-   * After this is called, the compute() code will no longer be called for
-   * this vertex unless a message is sent to it.  Then the compute() code
-   * will be called once again until this function is called.  The
-   * application finishes only when all vertices vote to halt.
+   * Add an edge for this vertex (happens immediately)
+   *
+   * @param edge Edge to add
    */
-  public void voteToHalt() {
-    halt = true;
-  }
+  void addEdge(Edge<I, E> edge);
 
   /**
-   * Re-activate vertex if halted.
+   * Removes all edges pointing to the given vertex id.
+   *
+   * @param targetVertexId the target vertex id
    */
-  public void wakeUp() {
-    halt = false;
-  }
+  void removeEdges(I targetVertexId);
 
   /**
-   * Is this vertex done?
-   *
-   * @return True if halted, false otherwise.
+   * If a {@link org.apache.giraph.edge.MutableEdgesWrapper} was used to
+   * provide a mutable iterator, 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 boolean isHalted() {
-    return halt;
-  }
+  void unwrapMutableEdges();
 
   /**
-   * Add an edge for this vertex (happens immediately)
-   *
-   * @param edge Edge to add
+   * Re-activate vertex if halted.
    */
-  public void addEdge(Edge<I, E> edge) {
-    edges.add(edge);
-  }
+  void wakeUp();
 
   /**
-   * Removes all edges pointing to the given vertex id.
+   * Is this vertex done?
    *
-   * @param targetVertexId the target vertex id
+   * @return True if halted, false otherwise.
    */
-  public void removeEdges(I targetVertexId) {
-    edges.remove(targetVertexId);
-  }
-
-  @Override
-  public String toString() {
-    return "Vertex(id=" + getId() + ",value=" + getValue() +
-        ",#edges=" + getNumEdges() + ")";
-  }
+  boolean isHalted();
 }
+

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java b/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
index 6f54dc7..18104cb 100644
--- a/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/VertexMutations.java
@@ -47,7 +47,8 @@ public class VertexMutations<I extends WritableComparable,
     V extends Writable, E extends Writable> implements VertexChanges<I, V, E>,
     Writable, ImmutableClassesGiraphConfigurable {
   /** List of added vertices during the last superstep */
-  private final List<Vertex<I, V, E>> addedVertexList = Lists.newArrayList();
+  private final List<Vertex<I, V, E>> addedVertexList =
+      Lists.newArrayList();
   /** Count of remove vertex requests */
   private int removedVertexCount = 0;
   /** List of added edges */

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
index 9695169..b8b82af 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexReader.java
@@ -18,7 +18,6 @@
 
 package org.apache.giraph.io;
 
-import java.io.IOException;
 import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.Writable;
@@ -26,6 +25,8 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import java.io.IOException;
+
 /**
  * Analogous to Hadoop's RecordReader for vertices.  Will read the
  * vertices from an input split.

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java b/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
index 70e721e..6a15ee8 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/VertexValueReader.java
@@ -18,13 +18,14 @@
 
 package org.apache.giraph.io;
 
-import java.io.IOException;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import java.io.IOException;
+
 /**
  * Vertex reader for {@link org.apache.giraph.io.VertexValueInputFormat}.
  *

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
index 112860c..cb0978f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/JsonLongDoubleFloatDoubleVertexInputFormat.java
@@ -17,7 +17,6 @@
  */
 package org.apache.giraph.io.formats;
 
-import com.google.common.collect.Lists;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
@@ -30,6 +29,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.json.JSONArray;
 import org.json.JSONException;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/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 91a19e6..121c18f 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
@@ -18,11 +18,6 @@
 
 package org.apache.giraph.io.formats;
 
-import com.google.common.collect.Sets;
-import java.io.IOException;
-import java.util.List;
-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.OutEdges;
@@ -36,6 +31,13 @@ import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Sets;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
 /**
  * This VertexInputFormat is meant for large scale testing.  It allows the user
  * to create an input data source that a variable number of aggregate vertices

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
index 1d31f4f..949fe3b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/SequenceFileVertexInputFormat.java
@@ -17,8 +17,6 @@
  */
 package org.apache.giraph.io.formats;
 
-import java.io.IOException;
-import java.util.List;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexReader;
@@ -30,6 +28,9 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 
+import java.io.IOException;
+import java.util.List;
+
 /**
  * Sequence file vertex input format based on {@link SequenceFileInputFormat}.
  *

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
index debdccc..5abd01c 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/formats/TextVertexInputFormat.java
@@ -18,8 +18,6 @@
 
 package org.apache.giraph.io.formats;
 
-import java.io.IOException;
-import java.util.List;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
@@ -33,6 +31,9 @@ import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import java.io.IOException;
+import java.util.List;
+
 /**
  * Abstract class that users should subclass to use their own text based
  * vertex input format.

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java b/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
index ca35c51..bf0c238 100644
--- a/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
+++ b/giraph-core/src/main/java/org/apache/giraph/io/iterables/VertexReaderWrapper.java
@@ -18,8 +18,6 @@
 
 package org.apache.giraph.io.iterables;
 
-import java.io.IOException;
-
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexReader;
@@ -28,6 +26,8 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import java.io.IOException;
+
 /**
  * Wraps {@link GiraphReader} for vertices into {@link VertexReader}
  *

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/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 dadce0f..0595897 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
@@ -368,8 +368,8 @@ public class DiskBackedPartitionStore<I extends WritableComparable,
    * @param vertex The vertex to serialize
    * @throws IOException
    */
-  private void writeVertexData(DataOutput output, Vertex<I, V, E> vertex)
-    throws IOException {
+  private void writeVertexData(DataOutput output,
+      Vertex<I, V, E> vertex) throws IOException {
     vertex.getId().write(output);
     vertex.getValue().write(output);
     output.writeBoolean(vertex.isHalted());

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java b/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
index aebd343..b6b9551 100644
--- a/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
+++ b/giraph-core/src/main/java/org/apache/giraph/partition/Partition.java
@@ -22,7 +22,6 @@ import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
-
 import org.apache.hadoop.util.Progressable;
 
 /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java b/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
index 6ca488c..d14f7a7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/InMemoryVertexInputFormat.java
@@ -18,10 +18,6 @@
 
 package org.apache.giraph.utils;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
 import org.apache.giraph.bsp.BspInputSplit;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexInputFormat;
@@ -32,6 +28,11 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
 /**
  * An input format that reads the input graph in memory. Used for unit tests.
  *

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java b/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
index 6e46a76..312a287 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/TestGraph.java
@@ -177,7 +177,6 @@ public class TestGraph<I extends WritableComparable,
    */
   protected Vertex<I, V, E> makeVertex(I id, V value,
       Entry<I, E>... edges) {
-    @SuppressWarnings("unchecked")
     Vertex<I, V, E> vertex = conf.createVertex();
     vertex.initialize(id, value, createEdges(edges));
     return vertex;

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/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 d0a6c46..b404646 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
@@ -232,7 +232,7 @@ public class TestVertexAndEdges {
     vertex.removeEdges(new LongWritable(500));
     assertEquals(999, vertex.getNumEdges());
     for (Edge<LongWritable, DoubleWritable> edge : vertex.getEdges()) {
-      assert(edge.getTargetVertexId().get() != 500);
+      assertTrue(edge.getTargetVertexId().get() != 500);
     }
 
     vertex.setEdgeValue(new LongWritable(10), new DoubleWritable(33.0));

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
index fdfb5e9..bd41460 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestAdjacencyListTextVertexOutputFormat.java
@@ -107,7 +107,7 @@ public class TestAdjacencyListTextVertexOutputFormat extends AdjacencyListTextVe
 
     when(vertex.getEdges()).thenReturn(cities);
 
-    RecordWriter<Text,Text> tw = mock(RecordWriter.class);
+    RecordWriter<Text, Text> tw = mock(RecordWriter.class);
     AdjacencyListTextVertexWriter writer = createVertexWriter(tw);
     writer.setConf(conf);
     writer.initialize(tac);
@@ -135,7 +135,7 @@ public class TestAdjacencyListTextVertexOutputFormat extends AdjacencyListTextVe
 
     when(vertex.getEdges()).thenReturn(cities);
 
-    RecordWriter<Text,Text> tw = mock(RecordWriter.class);
+    RecordWriter<Text, Text> tw = mock(RecordWriter.class);
     AdjacencyListTextVertexWriter writer = createVertexWriter(tw);
     writer.setConf(conf);
     writer.initialize(tac);

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
index c67a1c4..30cea2e 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestLongDoubleDoubleAdjacencyListVertexInputFormat.java
@@ -86,7 +86,7 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
     when(rr.getCurrentValue()).thenReturn(new Text(input));
     TextVertexReader vr = createVertexReader(rr);
 
-    vr.setConf( conf);
+    vr.setConf(conf);
     vr.initialize(null, tac);
 
     try {
@@ -105,7 +105,7 @@ public class TestLongDoubleDoubleAdjacencyListVertexInputFormat extends LongDoub
     when(rr.getCurrentValue()).thenReturn(new Text(input));
     TextVertexReader vr = createVertexReader(rr);
 
-    vr.setConf( conf);
+    vr.setConf(conf);
     vr.initialize(null, tac);
 
     try {

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java b/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
index 8034052..d34f258 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestTextDoubleDoubleAdjacencyListVertexInputFormat.java
@@ -19,12 +19,12 @@ package org.apache.giraph.io;
 
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.io.formats.AdjacencyListTextVertexInputFormat;
-import org.apache.giraph.io.formats.TextDoubleDoubleAdjacencyListVertexInputFormat;
-import org.apache.giraph.utils.EdgeIterables;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.AdjacencyListTextVertexInputFormat;
+import org.apache.giraph.io.formats.TextDoubleDoubleAdjacencyListVertexInputFormat;
+import org.apache.giraph.utils.EdgeIterables;
 import org.apache.giraph.utils.NoOpComputation;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.DoubleWritable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/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 8a1e9ed..be9db7e 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
@@ -20,9 +20,10 @@ package org.apache.giraph.partition;
 import org.apache.giraph.edge.ArrayListEdges;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.graph.DefaultVertex;
 import org.apache.giraph.graph.GiraphTransferRegulator;
-import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.job.GiraphJob;
 import org.apache.giraph.utils.NoOpComputation;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -46,7 +47,7 @@ public class TestGiraphTransferRegulator {
   private GiraphJob job;
   /** Instantiated vertex filled in from setup() */
   private Vertex<IntWritable, FloatWritable, DoubleWritable>
-      vertex = new Vertex<IntWritable, FloatWritable, DoubleWritable>();
+      vertex = new DefaultVertex<IntWritable, FloatWritable, DoubleWritable>();
 
   /**
    * Dummy vertex.

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java b/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
index 45542b9..0e68b56 100644
--- a/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
+++ b/giraph-core/src/test/java/org/apache/giraph/partition/TestPartitionStores.java
@@ -62,8 +62,7 @@ public class TestPartitionStores {
       Vertex<IntWritable, IntWritable, NullWritable>... vertices) {
     Partition<IntWritable, IntWritable, NullWritable> partition =
         conf.createPartition(id, context);
-    for (Vertex<IntWritable, IntWritable, NullWritable> v :
-        vertices) {
+    for (Vertex<IntWritable, IntWritable, NullWritable> v : vertices) {
       partition.putVertex(v);
     }
     return partition;

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java b/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
index d210928..bc5b5e2 100644
--- a/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
+++ b/giraph-core/src/test/java/org/apache/giraph/utils/MockUtils.java
@@ -22,14 +22,14 @@ import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.WorkerClientRequestProcessor;
 import org.apache.giraph.comm.messages.ByteArrayMessagesPerVertexStore;
-import org.apache.giraph.graph.Computation;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.ArrayListEdges;
+import org.apache.giraph.graph.Computation;
 import org.apache.giraph.graph.GraphState;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.partition.BasicPartitionOwner;
 import org.apache.giraph.partition.PartitionOwner;
-import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
index f56b4f6..18cc8bc 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleDoubleTextInputFormat.java
@@ -18,18 +18,19 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.collect.Lists;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.regex.Pattern;

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
index bfb5f40..b05a978 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/LongDoubleNullTextInputFormat.java
@@ -18,7 +18,6 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.collect.Lists;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.Edge;
@@ -31,6 +30,8 @@ import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.regex.Pattern;

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
index 5023a4e..2dc874e 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/NormalizingLongDoubleDoubleTextInputFormat.java
@@ -18,18 +18,19 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.collect.Lists;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
@@ -79,9 +80,8 @@ public class NormalizingLongDoubleDoubleTextInputFormat
     private final Pattern weightSeparator = Pattern.compile(":");
 
     @Override
-    public Vertex<LongWritable, DoubleWritable,
-        DoubleWritable> getCurrentVertex()
-      throws IOException, InterruptedException {
+    public Vertex<LongWritable, DoubleWritable, DoubleWritable>
+    getCurrentVertex() throws IOException, InterruptedException {
       Vertex<LongWritable, DoubleWritable, DoubleWritable> vertex =
           conf.createVertex();
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java
index d053bb3..f930bb0 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimplePageRankComputation.java
@@ -18,15 +18,12 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.List;
 import org.apache.giraph.aggregators.DoubleMaxAggregator;
 import org.apache.giraph.aggregators.DoubleMinAggregator;
 import org.apache.giraph.aggregators.LongSumAggregator;
-import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
@@ -41,6 +38,11 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+
 /**
  * Demonstrates the basic Pregel PageRank implementation.
  */
@@ -183,8 +185,8 @@ public class SimplePageRankComputation extends BasicComputation<LongWritable,
     }
 
     @Override
-    public Vertex<LongWritable, DoubleWritable,
-        FloatWritable> getCurrentVertex() throws IOException {
+    public Vertex<LongWritable, DoubleWritable, FloatWritable>
+    getCurrentVertex() throws IOException {
       Vertex<LongWritable, DoubleWritable, FloatWritable> vertex =
           getConf().createVertex();
       LongWritable vertexId = new LongWritable(

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java
index c3fd215..353f7df 100644
--- a/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java
+++ b/giraph-examples/src/main/java/org/apache/giraph/examples/SimpleSuperstepComputation.java
@@ -18,13 +18,9 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.BasicComputation;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.VertexReader;
 import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
@@ -37,6 +33,11 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+
 /**
  * Just a simple Vertex compute implementation that executes 3 supersteps, then
  * finishes.

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
index 28edbba..6c8a0b3 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/TestBspBasic.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java
index b94f8dc..dbcd569 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/ConnectedComponentsComputationTestInMemory.java
@@ -45,7 +45,7 @@ public class ConnectedComponentsComputationTestInMemory {
   public static Entry<IntWritable, NullWritable>[] makeEdges(int... args){
     Entry<IntWritable, NullWritable> result[] =
       new Entry[args.length];
-    for (int i=0; i<args.length; i++){
+    for (int i = 0; i < args.length; i++){
       result[i] = new SimpleEntry<IntWritable, NullWritable>(
           new IntWritable(args[i]), NullWritable.get());
     }
@@ -62,7 +62,7 @@ public class ConnectedComponentsComputationTestInMemory {
     conf.setCombinerClass(MinimumIntCombiner.class);
 
     TestGraph<IntWritable, IntWritable, NullWritable> graph =
-      new TestGraph<IntWritable, IntWritable, NullWritable> (conf);
+      new TestGraph<IntWritable, IntWritable, NullWritable>(conf);
     // a small graph with three components
     graph.addVertex(new IntWritable(1), new IntWritable(1), makeEdges(2, 3))
       .addVertex(new IntWritable(2), new IntWritable(2), makeEdges(1, 4, 5))
@@ -84,7 +84,7 @@ public class ConnectedComponentsComputationTestInMemory {
     TestGraph<IntWritable, IntWritable, NullWritable> results =
       InternalVertexRunner.run(conf, graph);
 
-    SetMultimap<Integer,Integer> components = parseResults(results);
+    SetMultimap<Integer, Integer> components = parseResults(results);
 
     Set<Integer> componentIDs = components.keySet();
     assertEquals(3, componentIDs.size());
@@ -115,9 +115,9 @@ public class ConnectedComponentsComputationTestInMemory {
     assertTrue(componentThree.contains(9));
   }
 
-  private SetMultimap<Integer,Integer> parseResults(
+  private SetMultimap<Integer, Integer> parseResults(
     TestGraph<IntWritable, IntWritable, NullWritable> results) {
-    SetMultimap<Integer,Integer> components = HashMultimap.create();
+    SetMultimap<Integer, Integer> components = HashMultimap.create();
     for (Vertex<IntWritable, IntWritable, NullWritable> vertex : results) {
       int component = vertex.getValue().get();
       components.put(component, vertex.getId().get());

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java
index 93a3248..6a74c21 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleShortestPathsComputationTest.java
@@ -21,6 +21,7 @@ package org.apache.giraph.examples;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.DefaultVertex;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexInputFormat;
 import org.apache.giraph.io.formats.JsonLongDoubleFloatDoubleVertexOutputFormat;
@@ -56,7 +57,7 @@ public class SimpleShortestPathsComputationTest {
   @Test
   public void testOnShorterPathFound() throws Exception {
     Vertex<LongWritable, DoubleWritable, FloatWritable> vertex =
-        new Vertex<LongWritable, DoubleWritable, FloatWritable>();
+        new DefaultVertex<LongWritable, DoubleWritable, FloatWritable>();
     SimpleShortestPathsComputation computation =
         new SimpleShortestPathsComputation();
     MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,
@@ -87,7 +88,7 @@ public class SimpleShortestPathsComputationTest {
   @Test
   public void testOnNoShorterPathFound() throws Exception {
     Vertex<LongWritable, DoubleWritable, FloatWritable> vertex =
-        new Vertex<LongWritable, DoubleWritable, FloatWritable>();
+        new DefaultVertex<LongWritable, DoubleWritable, FloatWritable>();
     SimpleShortestPathsComputation computation =
         new SimpleShortestPathsComputation();
     MockUtils.MockedEnvironment<LongWritable, DoubleWritable, FloatWritable,

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java
index 73516a4..7346745 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/examples/SimpleTriangleClosingComputationTest.java
@@ -18,17 +18,17 @@
 
 package org.apache.giraph.examples;
 
-import com.google.common.collect.Lists;
-
+import org.apache.giraph.edge.EdgeFactory;
+import org.apache.giraph.graph.DefaultVertex;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.utils.MockUtils;
-import org.apache.giraph.edge.EdgeFactory;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.junit.Test;
 
-import static org.apache.giraph.examples.SimpleTriangleClosingComputation.IntArrayListWritable;
+import com.google.common.collect.Lists;
 
+import static org.apache.giraph.examples.SimpleTriangleClosingComputation.IntArrayListWritable;
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -44,7 +44,7 @@ public class SimpleTriangleClosingComputationTest {
   public void testSuperstepZero() throws Exception {
     // this guy should end up with an array value of 4
     Vertex<IntWritable, IntArrayListWritable, NullWritable> vertex =
-        new Vertex<IntWritable, IntArrayListWritable, NullWritable>();
+        new DefaultVertex<IntWritable, IntArrayListWritable, NullWritable>();
 
     IntArrayListWritable alw = new IntArrayListWritable();
 
@@ -71,7 +71,7 @@ public class SimpleTriangleClosingComputationTest {
     // see if the vertex interprets its incoming
     // messages properly to verify the algorithm
     Vertex<IntWritable, IntArrayListWritable, NullWritable> vertex =
-        new Vertex<IntWritable, IntArrayListWritable, NullWritable>();
+        new DefaultVertex<IntWritable, IntArrayListWritable, NullWritable>();
     SimpleTriangleClosingComputation computation =
         new SimpleTriangleClosingComputation();
     MockUtils.MockedEnvironment env = MockUtils.prepareVertexAndComputation(

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
index 1af46d1..f547172 100644
--- a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
+++ b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
@@ -17,12 +17,11 @@
  */
 package org.apache.giraph.io.hbase.edgemarker;
 
-import com.google.common.collect.Lists;
-import org.apache.giraph.io.VertexReader;
-import org.apache.giraph.io.hbase.HBaseVertexInputFormat;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.VertexReader;
+import org.apache.giraph.io.hbase.HBaseVertexInputFormat;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Text;
@@ -30,6 +29,8 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Lists;
+
 import java.io.IOException;
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
index a04d386..aa95f96 100644
--- a/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
+++ b/giraph-hbase/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
@@ -66,7 +66,7 @@ public class TableEdgeOutputFormat
               byte[] rowBytes = vertex.getId().getBytes();
               Put put = new Put(rowBytes);
               Text value = vertex.getValue();
-              if(value.toString().length() > 0)   {
+              if (value.toString().length() > 0)   {
                  put.add(CF, PARENT, value.getBytes());
                  writer.write(new ImmutableBytesWritable(rowBytes), put);
               }

http://git-wip-us.apache.org/repos/asf/giraph/blob/90f3116e/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
index 53c7eed..edecbff 100644
--- a/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
+++ b/giraph-hcatalog/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
@@ -18,9 +18,6 @@
 
 package org.apache.giraph.io.hcatalog;
 
-import com.google.common.collect.Lists;
-import java.io.IOException;
-import java.util.List;
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.EdgeFactory;
 import org.apache.giraph.graph.Vertex;
@@ -36,6 +33,11 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hcatalog.data.HCatRecord;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.List;
+
 /**
  * Abstract class that users should subclass to load data from a Hive or Pig
  * table. You can easily implement a {@link HCatalogVertexReader} by extending