giraph-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ach...@apache.org
Subject svn commit: r1365352 [3/4] - in /giraph/trunk: ./ giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/ giraph-formats-contrib/src/test/java/org/apache/giraph/format/accumulo/edgemarker/ giraph-formats-contrib/src/test/java/org/apache...
Date Tue, 24 Jul 2012 23:37:45 GMT
Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java Tue Jul 24 23:37:42 2012
@@ -18,8 +18,6 @@
 
 package org.apache.giraph.graph;
 
-import com.google.common.collect.Iterables;
-
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.graph.partition.Partition;
@@ -36,6 +34,8 @@ import org.apache.hadoop.io.WritableComp
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Iterables;
+
 import java.io.IOException;
 import java.lang.reflect.Type;
 import java.net.URL;
@@ -43,7 +43,6 @@ import java.net.URLDecoder;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Enumeration;
-import java.util.Iterator;
 import java.util.List;
 
 /**
@@ -186,10 +185,10 @@ public class GraphMapper<I extends Writa
    * @param conf Configuration to get the various classes
    */
   public void determineClassTypes(Configuration conf) {
-    Class<? extends BasicVertex<I, V, E, M>> vertexClass =
+    Class<? extends Vertex<I, V, E, M>> vertexClass =
         BspUtils.<I, V, E, M>getVertexClass(conf);
-    List<Class<?>> classList = ReflectionUtils.<BasicVertex>getTypeArguments(
-        BasicVertex.class, vertexClass);
+    List<Class<?>> classList = ReflectionUtils.<Vertex>getTypeArguments(
+        Vertex.class, vertexClass);
     Type vertexIndexType = classList.get(0);
     Type vertexValueType = classList.get(1);
     Type edgeValueType = classList.get(2);
@@ -309,7 +308,7 @@ public class GraphMapper<I extends Writa
               "vertex - " + edgeValueType +
               ", vertex resolver - " + classList.get(3));
     }
-    conf.setClass(GiraphJob.VERTEX_INDEX_CLASS,
+    conf.setClass(GiraphJob.VERTEX_ID_CLASS,
         (Class<?>) vertexIndexType,
         WritableComparable.class);
     conf.setClass(GiraphJob.VERTEX_VALUE_CLASS,
@@ -506,7 +505,7 @@ public class GraphMapper<I extends Writa
     if (done) {
       return;
     }
-    if ((serviceWorker != null) && (graphState.getNumVertices() == 0)) {
+    if ((serviceWorker != null) && (graphState.getTotalNumVertices() == 0)) {
       return;
     }
 
@@ -585,28 +584,26 @@ public class GraphMapper<I extends Writa
       for (Partition<I, V, E, M> partition :
         serviceWorker.getPartitionMap().values()) {
         PartitionStats partitionStats =
-            new PartitionStats(partition.getPartitionId(), 0, 0, 0);
-        for (BasicVertex<I, V, E, M> basicVertex :
+            new PartitionStats(partition.getId(), 0, 0, 0);
+        for (Vertex<I, V, E, M> vertex :
           partition.getVertices()) {
           // Make sure every vertex has the current
           // graphState before computing
-          basicVertex.setGraphState(graphState);
-          if (basicVertex.isHalted() &
-              !Iterables.isEmpty(basicVertex.getMessages())) {
-            basicVertex.halt = false;
+          vertex.setGraphState(graphState);
+          if (vertex.isHalted() &
+              !Iterables.isEmpty(vertex.getMessages())) {
+            vertex.wakeUp();
           }
-          if (!basicVertex.isHalted()) {
-            Iterator<M> vertexMsgIt =
-                basicVertex.getMessages().iterator();
+          if (!vertex.isHalted()) {
             context.progress();
-            basicVertex.compute(vertexMsgIt);
-            basicVertex.releaseResources();
+            vertex.compute(vertex.getMessages());
+            vertex.releaseResources();
           }
-          if (basicVertex.isHalted()) {
+          if (vertex.isHalted()) {
             partitionStats.incrFinishedVertexCount();
           }
           partitionStats.incrVertexCount();
-          partitionStats.addEdgeCount(basicVertex.getNumOutEdges());
+          partitionStats.addEdgeCount(vertex.getNumEdges());
         }
         partitionStatsList.add(partitionStats);
       }

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/GraphState.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphState.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/GraphState.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/GraphState.java Tue Jul 24 23:37:42 2012
@@ -62,7 +62,7 @@ E extends Writable, M extends Writable> 
     return this;
   }
 
-  public long getNumVertices() {
+  public long getTotalNumVertices() {
     return numVertices;
   }
 
@@ -72,12 +72,12 @@ E extends Writable, M extends Writable> 
    * @param numVertices Current number of vertices.
    * @return Returns this object.
    */
-  public GraphState<I, V, E, M> setNumVertices(long numVertices) {
+  public GraphState<I, V, E, M> setTotalNumVertices(long numVertices) {
     this.numVertices = numVertices;
     return this;
   }
 
-  public long getNumEdges() {
+  public long getTotalNumEdges() {
     return numEdges;
   }
 
@@ -87,7 +87,7 @@ E extends Writable, M extends Writable> 
    * @param numEdges Current number of edges.
    * @return Returns this object.
    */
-  public GraphState<I, V, E, M> setNumEdges(long numEdges) {
+  public GraphState<I, V, E, M> setTotalNumEdges(long numEdges) {
     this.numEdges = numEdges;
     return this;
   }

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/HashMapVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/HashMapVertex.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/HashMapVertex.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/HashMapVertex.java Tue Jul 24 23:37:42 2012
@@ -22,14 +22,15 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
 
+import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -53,39 +54,27 @@ public abstract class HashMapVertex<I ex
     extends MutableVertex<I, V, E, M> {
   /** Class logger */
   private static final Logger LOG = Logger.getLogger(HashMapVertex.class);
-  /** Map of destination vertices and their edge values */
-  protected final Map<I, E> destEdgeMap =
-      new HashMap<I, E>();
-  /** Vertex id */
-  private I vertexId = null;
-  /** Vertex value */
-  private V vertexValue = null;
+  /** Map of target vertices and their edge values */
+  protected Map<I, E> edgeMap = new HashMap<I, E>();
   /** List of incoming messages from the previous superstep */
-  private final List<M> msgList = Lists.newArrayList();
+  private List<M> messageList = Lists.newArrayList();
 
   @Override
   public void initialize(
-      I vertexId, V vertexValue, Map<I, E> edges, Iterable<M> messages) {
-    if (vertexId != null) {
-      setVertexId(vertexId);
-    }
-    if (vertexValue != null) {
-      setVertexValue(vertexValue);
-    }
-    if (edges != null) {
-      destEdgeMap.putAll(edges);
-    }
+      I id, V value, Map<I, E> edges, Iterable<M> messages) {
+    super.initialize(id, value);
+    edgeMap.putAll(edges);
     if (messages != null) {
-      Iterables.<M>addAll(msgList, messages);
+      Iterables.<M>addAll(messageList, messages);
     }
   }
 
   @Override
-  public final boolean addEdge(I targetVertexId, E edgeValue) {
-    if (destEdgeMap.put(targetVertexId, edgeValue) != null) {
+  public final boolean addEdge(I targetVertexId, E value) {
+    if (edgeMap.put(targetVertexId, value) != null) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("addEdge: Vertex=" + vertexId +
-            ": already added an edge value for dest vertex id " +
+        LOG.debug("addEdge: Vertex=" + getId() +
+            ": already added an edge value for target vertex id " +
             targetVertexId);
       }
       return false;
@@ -95,145 +84,123 @@ public abstract class HashMapVertex<I ex
   }
 
   @Override
-  public long getSuperstep() {
-    return getGraphState().getSuperstep();
-  }
-
-  @Override
-  public final void setVertexId(I vertexId) {
-    this.vertexId = vertexId;
-  }
-
-  @Override
-  public final I getVertexId() {
-    return vertexId;
+  public boolean hasEdge(I targetVertexId) {
+    return edgeMap.containsKey(targetVertexId);
   }
 
+  /**
+   * Get an iterator to the edges on this vertex.
+   *
+   * @return A <em>sorted</em> iterator, as defined by the sort-order
+   *         of the vertex ids
+   */
   @Override
-  public final V getVertexValue() {
-    return vertexValue;
+  public Iterable<Edge<I, E>> getEdges() {
+    return Iterables.transform(edgeMap.entrySet(),
+        new Function<Map.Entry<I, E>, Edge<I, E>>() {
+
+          @Override
+          public Edge<I, E> apply(Map.Entry<I, E> edge) {
+            return new Edge<I, E>(edge.getKey(), edge.getValue());
+          }
+        });
   }
 
   @Override
-  public final void setVertexValue(V vertexValue) {
-    this.vertexValue = vertexValue;
+  public E getEdgeValue(I targetVertexId) {
+    return edgeMap.get(targetVertexId);
   }
 
   @Override
-  public E getEdgeValue(I targetVertexId) {
-    return destEdgeMap.get(targetVertexId);
+  public int getNumEdges() {
+    return edgeMap.size();
   }
 
   @Override
-  public boolean hasEdge(I targetVertexId) {
-    return destEdgeMap.containsKey(targetVertexId);
+  public E removeEdge(I targetVertexId) {
+    return edgeMap.remove(targetVertexId);
   }
 
-  /**
-   * Get an iterator to the edges on this vertex.
-   *
-   * @return A <em>sorted</em> iterator, as defined by the sort-order
-   *         of the vertex ids
-   */
   @Override
-  public Iterator<I> getOutEdgesIterator() {
-    return destEdgeMap.keySet().iterator();
+  public final void sendMessageToAllEdges(M message) {
+    for (I targetVertexId : edgeMap.keySet()) {
+      sendMessage(targetVertexId, message);
+    }
   }
 
   @Override
-  public int getNumOutEdges() {
-    return destEdgeMap.size();
+  void putMessages(Iterable<M> messages) {
+    messageList.clear();
+    Iterables.addAll(messageList, messages);
   }
 
   @Override
-  public E removeEdge(I targetVertexId) {
-    return destEdgeMap.remove(targetVertexId);
+  public Iterable<M> getMessages() {
+    return Iterables.unmodifiableIterable(messageList);
   }
 
   @Override
-  public final void sendMsgToAllEdges(M msg) {
-    if (msg == null) {
-      throw new IllegalArgumentException(
-          "sendMsgToAllEdges: Cannot send null message to all edges");
-    }
-    for (I targetVertexId : destEdgeMap.keySet()) {
-      sendMsg(targetVertexId, msg);
-    }
+  public int getNumMessages() {
+    return messageList.size();
   }
 
   @Override
   public final void readFields(DataInput in) throws IOException {
-    vertexId = BspUtils.<I>createVertexIndex(getConf());
+    I vertexId = BspUtils.<I>createVertexId(getConf());
     vertexId.readFields(in);
-    boolean hasVertexValue = in.readBoolean();
-    if (hasVertexValue) {
-      vertexValue = BspUtils.<V>createVertexValue(getConf());
-      vertexValue.readFields(in);
-    }
-    long edgeMapSize = in.readLong();
-    for (long i = 0; i < edgeMapSize; ++i) {
-      I targetVertexId = BspUtils.<I>createVertexIndex(getConf());
+    V vertexValue = BspUtils.<V>createVertexValue(getConf());
+    vertexValue.readFields(in);
+    super.initialize(vertexId, vertexValue);
+
+    int numEdges = in.readInt();
+    edgeMap = Maps.newHashMapWithExpectedSize(numEdges);
+    for (int i = 0; i < numEdges; ++i) {
+      I targetVertexId = BspUtils.<I>createVertexId(getConf());
       targetVertexId.readFields(in);
       E edgeValue = BspUtils.<E>createEdgeValue(getConf());
       edgeValue.readFields(in);
-      addEdge(targetVertexId, edgeValue);
+      edgeMap.put(targetVertexId, edgeValue);
     }
-    long msgListSize = in.readLong();
-    for (long i = 0; i < msgListSize; ++i) {
-      M msg = BspUtils.<M>createMessageValue(getConf());
-      msg.readFields(in);
-      msgList.add(msg);
+
+    int numMessages = in.readInt();
+    messageList = Lists.newArrayListWithCapacity(numMessages);
+    for (int i = 0; i < numMessages; ++i) {
+      M message = BspUtils.<M>createMessageValue(getConf());
+      message.readFields(in);
+      messageList.add(message);
+    }
+
+    boolean halt = in.readBoolean();
+    if (halt) {
+      voteToHalt();
+    } else {
+      wakeUp();
     }
-    halt = in.readBoolean();
   }
 
   @Override
   public final void write(DataOutput out) throws IOException {
-    vertexId.write(out);
-    out.writeBoolean(vertexValue != null);
-    if (vertexValue != null) {
-      vertexValue.write(out);
-    }
-    out.writeLong(destEdgeMap.size());
-    for (Map.Entry<I, E> edge : destEdgeMap.entrySet()) {
+    getId().write(out);
+    getValue().write(out);
+
+    out.writeInt(edgeMap.size());
+    for (Map.Entry<I, E> edge : edgeMap.entrySet()) {
       edge.getKey().write(out);
       edge.getValue().write(out);
     }
-    out.writeLong(msgList.size());
-    for (M msg : msgList) {
-      msg.write(out);
-    }
-    out.writeBoolean(halt);
-  }
 
-  @Override
-  void putMessages(Iterable<M> messages) {
-    msgList.clear();
-    for (M message : messages) {
-      msgList.add(message);
+    out.writeInt(messageList.size());
+    for (M message : messageList) {
+      message.write(out);
     }
-  }
 
-  @Override
-  public Iterable<M> getMessages() {
-    return Iterables.unmodifiableIterable(msgList);
-  }
-
-  @Override
-  public int getNumMessages() {
-    return msgList.size();
+    out.writeBoolean(isHalted());
   }
 
   @Override
   void releaseResources() {
     // Hint to GC to free the messages
-    msgList.clear();
-  }
-
-  @Override
-  public String toString() {
-    return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() +
-        ",#edges=" + destEdgeMap.size() + ")";
+    messageList.clear();
   }
 }
 

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/IntIntNullIntVertex.java Tue Jul 24 23:37:42 2012
@@ -18,11 +18,12 @@
 
 package org.apache.giraph.graph;
 
-import com.google.common.collect.Iterables;
 import org.apache.giraph.utils.UnmodifiableIntArrayIterator;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 
+import com.google.common.collect.Iterables;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -30,32 +31,27 @@ import java.util.Iterator;
 import java.util.Map;
 
 /**
- * Simple implementation of {@link BasicVertex} using an int as id, value and
+ * Simple implementation of {@link Vertex} using an int as id, value and
  * message.  Edges are immutable and unweighted. This class aims to be as
  * memory efficient as possible.
  */
 public abstract class IntIntNullIntVertex extends
-    BasicVertex<IntWritable, IntWritable, NullWritable, IntWritable> {
-  /** Int represented vertex id */
-  private int id;
-  /** Int represented vertex value */
-  private int value;
+    SimpleVertex<IntWritable, IntWritable, IntWritable> {
   /** Int array of neighbor vertex ids */
   private int[] neighbors;
   /** Int array of messages */
   private int[] messages;
 
   @Override
-  public void initialize(IntWritable vertexId, IntWritable vertexValue,
-      Map<IntWritable, NullWritable> edges,
-      Iterable<IntWritable> messages) {
-    id = vertexId.get();
-    value = vertexValue.get();
+  public void initialize(IntWritable id, IntWritable value,
+                         Map<IntWritable, NullWritable> edges,
+                         Iterable<IntWritable> messages) {
+    super.initialize(id, value);
     this.neighbors = new int[(edges != null) ? edges.size() : 0];
     int n = 0;
     if (edges != null) {
-      for (IntWritable neighbor : edges.keySet()) {
-        this.neighbors[n++] = neighbor.get();
+      for (Map.Entry<IntWritable, NullWritable> edge : edges.entrySet()) {
+        this.neighbors[n++] = edge.getKey().get();
       }
     }
     this.messages = new int[(messages != null) ? Iterables.size(messages) : 0];
@@ -68,28 +64,13 @@ public abstract class IntIntNullIntVerte
   }
 
   @Override
-  public IntWritable getVertexId() {
-    return new IntWritable(id);
-  }
-
-  @Override
-  public IntWritable getVertexValue() {
-    return new IntWritable(value);
-  }
-
-  @Override
-  public void setVertexValue(IntWritable vertexValue) {
-    value = vertexValue.get();
-  }
-
-  @Override
-  public Iterator<IntWritable> getOutEdgesIterator() {
-    return new UnmodifiableIntArrayIterator(neighbors);
-  }
-
-  @Override
-  public NullWritable getEdgeValue(IntWritable targetVertexId) {
-    return NullWritable.get();
+  public Iterable<IntWritable> getNeighbors() {
+    return new Iterable<IntWritable>() {
+      @Override
+      public Iterator<IntWritable> iterator() {
+        return new UnmodifiableIntArrayIterator(neighbors);
+      }
+    };
   }
 
   @Override
@@ -103,18 +84,11 @@ public abstract class IntIntNullIntVerte
   }
 
   @Override
-  public int getNumOutEdges() {
+  public int getNumEdges() {
     return neighbors.length;
   }
 
   @Override
-  public void sendMsgToAllEdges(final IntWritable message) {
-    for (int neighbor : neighbors) {
-      sendMsg(new IntWritable(neighbor), message);
-    }
-  }
-
-  @Override
   public Iterable<IntWritable> getMessages() {
     return new Iterable<IntWritable>() {
       @Override
@@ -145,8 +119,8 @@ public abstract class IntIntNullIntVerte
 
   @Override
   public void write(final DataOutput out) throws IOException {
-    out.writeInt(id);
-    out.writeInt(value);
+    out.writeInt(getId().get());
+    out.writeInt(getValue().get());
     out.writeInt(neighbors.length);
     for (int n = 0; n < neighbors.length; n++) {
       out.writeInt(neighbors[n]);
@@ -155,13 +129,14 @@ public abstract class IntIntNullIntVerte
     for (int n = 0; n < messages.length; n++) {
       out.writeInt(messages[n]);
     }
-    out.writeBoolean(halt);
+    out.writeBoolean(isHalted());
   }
 
   @Override
   public void readFields(DataInput in) throws IOException {
-    id = in.readInt();
-    value = in.readInt();
+    int id = in.readInt();
+    int value = in.readInt();
+    super.initialize(new IntWritable(id), new IntWritable(value));
     int numEdges = in.readInt();
     neighbors = new int[numEdges];
     for (int n = 0; n < numEdges; n++) {
@@ -172,7 +147,11 @@ public abstract class IntIntNullIntVerte
     for (int n = 0; n < numMessages; n++) {
       messages[n] = in.readInt();
     }
-    halt = in.readBoolean();
+    boolean halt = in.readBoolean();
+    if (halt) {
+      voteToHalt();
+    } else {
+      wakeUp();
+    }
   }
-
 }

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/LongDoubleFloatDoubleVertex.java Tue Jul 24 23:37:42 2012
@@ -23,7 +23,6 @@ import org.apache.hadoop.io.LongWritable
 import org.apache.log4j.Logger;
 import org.apache.mahout.math.function.DoubleProcedure;
 import org.apache.mahout.math.function.LongFloatProcedure;
-import org.apache.mahout.math.function.LongProcedure;
 import org.apache.mahout.math.list.DoubleArrayList;
 import org.apache.mahout.math.map.OpenLongFloatHashMap;
 
@@ -41,39 +40,30 @@ import java.util.Map;
  */
 public abstract class LongDoubleFloatDoubleVertex extends
     MutableVertex<LongWritable, DoubleWritable, FloatWritable,
-    DoubleWritable> {
+        DoubleWritable> {
   /** Class logger */
   private static final Logger LOG =
       Logger.getLogger(LongDoubleFloatDoubleVertex.class);
-  /** Long vertex id */
-  private long vertexId;
-  /** Double vertex value */
-  private double vertexValue;
   /** Stores the edges */
-  private OpenLongFloatHashMap verticesWithEdgeValues =
+  private OpenLongFloatHashMap edgeMap =
       new OpenLongFloatHashMap();
   /** Message list storage */
   private DoubleArrayList messageList = new DoubleArrayList();
 
   @Override
-  public void initialize(LongWritable vertexIdW, DoubleWritable vertexValueW,
-      Map<LongWritable, FloatWritable> edgesW,
-      Iterable<DoubleWritable> messagesW) {
-    if (vertexIdW != null) {
-      vertexId = vertexIdW.get();
-    }
-    if (vertexValueW != null) {
-      vertexValue = vertexValueW.get();
-    }
-    if (edgesW != null) {
-      for (Map.Entry<LongWritable, FloatWritable> entry :
-        edgesW.entrySet()) {
-        verticesWithEdgeValues.put(entry.getKey().get(),
-            entry.getValue().get());
+  public void initialize(LongWritable id, DoubleWritable value,
+                         Map<LongWritable, FloatWritable> edges,
+                         Iterable<DoubleWritable> messages) {
+    super.initialize(id, value);
+    if (edges != null) {
+      for (Map.Entry<LongWritable, FloatWritable> edge :
+        edges.entrySet()) {
+        edgeMap.put(edge.getKey().get(),
+            edge.getValue().get());
       }
     }
-    if (messagesW != null) {
-      for (DoubleWritable m : messagesW) {
+    if (messages != null) {
+      for (DoubleWritable m : messages) {
         messageList.add(m.get());
       }
     }
@@ -82,9 +72,9 @@ public abstract class LongDoubleFloatDou
   @Override
   public final boolean addEdge(LongWritable targetId,
       FloatWritable edgeValue) {
-    if (verticesWithEdgeValues.put(targetId.get(), edgeValue.get())) {
+    if (edgeMap.put(targetId.get(), edgeValue.get())) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("addEdge: Vertex=" + vertexId +
+        LOG.debug("addEdge: Vertex=" + getId() +
             ": already added an edge value for dest vertex id " +
             targetId.get());
       }
@@ -97,9 +87,9 @@ public abstract class LongDoubleFloatDou
   @Override
   public FloatWritable removeEdge(LongWritable targetVertexId) {
     long target = targetVertexId.get();
-    if (verticesWithEdgeValues.containsKey(target)) {
-      float value = verticesWithEdgeValues.get(target);
-      verticesWithEdgeValues.removeKey(target);
+    if (edgeMap.containsKey(target)) {
+      float value = edgeMap.get(target);
+      edgeMap.removeKey(target);
       return new FloatWritable(value);
     } else {
       return null;
@@ -107,127 +97,78 @@ public abstract class LongDoubleFloatDou
   }
 
   @Override
-  public final void setVertexId(LongWritable vertexId) {
-    this.vertexId = vertexId.get();
-  }
-
-  @Override
-  public final LongWritable getVertexId() {
-    // TODO: possibly not make new objects every time?
-    return new LongWritable(vertexId);
-  }
-
-  @Override
-  public final DoubleWritable getVertexValue() {
-    return new DoubleWritable(vertexValue);
-  }
-
-  @Override
-  public final void setVertexValue(DoubleWritable vertexValue) {
-    this.vertexValue = vertexValue.get();
-  }
-
-  @Override
-  public final void sendMsg(LongWritable id, DoubleWritable msg) {
-    if (msg == null) {
-      throw new IllegalArgumentException(
-          "sendMsg: Cannot send null message to " + id);
-    }
-    getGraphState().getWorkerCommunications().sendMessageReq(id, msg);
-  }
+  public Iterable<Edge<LongWritable, FloatWritable>> getEdges() {
+    final long[] targetVertices = edgeMap.keys().elements();
+    final int numEdges = edgeMap.size();
 
-  @Override
-  public final void sendMsgToAllEdges(final DoubleWritable msg) {
-    if (msg == null) {
-      throw new IllegalArgumentException(
-          "sendMsgToAllEdges: Cannot send null message to all edges");
-    }
-    final MutableVertex<LongWritable, DoubleWritable, FloatWritable,
-    DoubleWritable> vertex = this;
-    verticesWithEdgeValues.forEachKey(new LongProcedure() {
+    return new Iterable<Edge<LongWritable, FloatWritable>>() {
       @Override
-      public boolean apply(long destVertexId) {
-        vertex.sendMsg(new LongWritable(destVertexId), msg);
-        return true;
-      }
-    });
-  }
-
-  @Override
-  public long getNumVertices() {
-    return getGraphState().getNumVertices();
-  }
-
-  @Override
-  public long getNumEdges() {
-    return getGraphState().getNumEdges();
-  }
-
-  @Override
-  public Iterator<LongWritable> getOutEdgesIterator() {
-    final long[] destVertices = verticesWithEdgeValues.keys().elements();
-    final int destVerticesSize = verticesWithEdgeValues.size();
-    return new Iterator<LongWritable>() {
-      private int offset = 0;
-      @Override public boolean hasNext() {
-        return offset < destVerticesSize;
-      }
-
-      @Override public LongWritable next() {
-        return new LongWritable(destVertices[offset++]);
-      }
-
-      @Override public void remove() {
-        throw new UnsupportedOperationException(
-            "Mutation disallowed for edge list via iterator");
+      public Iterator<Edge<LongWritable, FloatWritable>> iterator() {
+        return new Iterator<Edge<LongWritable, FloatWritable>>() {
+          private int offset = 0;
+
+          @Override
+          public boolean hasNext() {
+            return offset < numEdges;
+          }
+
+          @Override
+          public Edge<LongWritable, FloatWritable> next() {
+            long targetVertex = targetVertices[offset++];
+            return new Edge<LongWritable, FloatWritable>(
+                new LongWritable(targetVertex),
+                new FloatWritable(targetVertex));
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException(
+                "Mutation disallowed for edge list via iterator");
+          }
+        };
       }
     };
   }
 
   @Override
-  public FloatWritable getEdgeValue(LongWritable targetVertexId) {
-    return new FloatWritable(
-        verticesWithEdgeValues.get(targetVertexId.get()));
-  }
-
-  @Override
   public boolean hasEdge(LongWritable targetVertexId) {
-    return verticesWithEdgeValues.containsKey(targetVertexId.get());
+    return edgeMap.containsKey(targetVertexId.get());
   }
 
   @Override
-  public int getNumOutEdges() {
-    return verticesWithEdgeValues.size();
-  }
-
-  @Override
-  public long getSuperstep() {
-    return getGraphState().getSuperstep();
+  public int getNumEdges() {
+    return edgeMap.size();
   }
 
   @Override
   public final void readFields(DataInput in) throws IOException {
-    vertexId = in.readLong();
-    vertexValue = in.readDouble();
+    long id = in.readLong();
+    double value = in.readDouble();
+    super.initialize(new LongWritable(id), new DoubleWritable(value));
     long edgeMapSize = in.readLong();
     for (long i = 0; i < edgeMapSize; ++i) {
-      long destVertexId = in.readLong();
+      long targetVertexId = in.readLong();
       float edgeValue = in.readFloat();
-      verticesWithEdgeValues.put(destVertexId, edgeValue);
+      edgeMap.put(targetVertexId, edgeValue);
     }
-    long msgListSize = in.readLong();
-    for (long i = 0; i < msgListSize; ++i) {
+    long messageListSize = in.readLong();
+    for (long i = 0; i < messageListSize; ++i) {
       messageList.add(in.readDouble());
     }
-    halt = in.readBoolean();
+    boolean halt = in.readBoolean();
+    if (halt) {
+      voteToHalt();
+    } else {
+      wakeUp();
+    }
   }
 
   @Override
   public final void write(final DataOutput out) throws IOException {
-    out.writeLong(vertexId);
-    out.writeDouble(vertexValue);
-    out.writeLong(verticesWithEdgeValues.size());
-    verticesWithEdgeValues.forEachPair(new LongFloatProcedure() {
+    out.writeLong(getId().get());
+    out.writeDouble(getValue().get());
+    out.writeLong(edgeMap.size());
+    edgeMap.forEachPair(new LongFloatProcedure() {
       @Override
       public boolean apply(long destVertexId, float edgeValue) {
         try {
@@ -253,7 +194,7 @@ public abstract class LongDoubleFloatDou
         return true;
       }
     });
-    out.writeBoolean(halt);
+    out.writeBoolean(isHalted());
   }
 
   @Override
@@ -271,19 +212,13 @@ public abstract class LongDoubleFloatDou
   }
 
   @Override
-  public Iterable<DoubleWritable> getMessages() {
-    return new UnmodifiableDoubleWritableIterable(messageList);
-  }
-
-  @Override
   public int getNumMessages() {
     return messageList.size();
   }
 
   @Override
-  public String toString() {
-    return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() +
-        ",#edges=" + getNumOutEdges() + ")";
+  public Iterable<DoubleWritable> getMessages() {
+    return new UnmodifiableDoubleWritableIterable(messageList);
   }
 
   /**

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/MasterCompute.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/MasterCompute.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/MasterCompute.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/MasterCompute.java Tue Jul 24 23:37:42 2012
@@ -43,7 +43,7 @@ import org.apache.hadoop.mapreduce.Mappe
 public abstract class MasterCompute implements AggregatorUsage, Writable,
     Configurable {
   /** If true, do not do anymore computation on this vertex. */
-  protected boolean halt = false;
+  private boolean halt = false;
   /** Global graph state **/
   private GraphState graphState;
   /** Configuration */
@@ -76,8 +76,8 @@ public abstract class MasterCompute impl
    *
    * @return Total number of vertices (-1 if first superstep)
    */
-  public long getNumVertices() {
-    return getGraphState().getNumVertices();
+  public long getTotalNumVertices() {
+    return getGraphState().getTotalNumVertices();
   }
 
   /**
@@ -86,8 +86,8 @@ public abstract class MasterCompute impl
    *
    * @return Total number of edges (-1 if first superstep)
    */
-  public long getNumEdges() {
-    return getGraphState().getNumEdges();
+  public long getTotalNumEdges() {
+    return getGraphState().getTotalNumEdges();
   }
 
   /**

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java Tue Jul 24 23:37:42 2012
@@ -36,22 +36,15 @@ import java.util.Map;
 @SuppressWarnings("rawtypes")
 public abstract class MutableVertex<I extends WritableComparable,
     V extends Writable, E extends Writable, M extends Writable>
-    extends BasicVertex<I, V, E, M> {
-  /**
-   * Set the vertex id
-   *
-   * @param id Vertex id is set to this (instantiated by the user)
-   */
-  public abstract void setVertexId(I id);
-
+    extends Vertex<I, V, E, M> {
   /**
    * Add an edge for this vertex (happens immediately)
    *
    * @param targetVertexId target vertex
-   * @param edgeValue value of the edge
+   * @param value value of the edge
    * @return Return true if succeeded, false otherwise
    */
-  public abstract boolean addEdge(I targetVertexId, E edgeValue);
+  public abstract boolean addEdge(I targetVertexId, E value);
 
   /**
    * Removes an edge for this vertex (happens immediately).
@@ -72,7 +65,7 @@ public abstract class MutableVertex<I ex
    * @param messages Messages to be added to the vertex (typically empty)
    * @return A new vertex for adding to the graph
    */
-  public BasicVertex<I, V, E, M> instantiateVertex(
+  public Vertex<I, V, E, M> instantiateVertex(
       I vertexId, V vertexValue, Map<I, E> edges, Iterable<M> messages) {
     MutableVertex<I, V, E, M> mutableVertex =
         (MutableVertex<I, V, E, M>) BspUtils
@@ -88,10 +81,10 @@ public abstract class MutableVertex<I ex
    *
    * @param vertex User created vertex
    */
-  public void addVertexRequest(BasicVertex<I, V, E, M> vertex)
+  public void addVertexRequest(Vertex<I, V, E, M> vertex)
     throws IOException {
     getGraphState().getWorkerCommunications().
-    addVertexReq(vertex);
+        addVertexRequest(vertex);
   }
 
   /**
@@ -102,7 +95,7 @@ public abstract class MutableVertex<I ex
    */
   public void removeVertexRequest(I vertexId) throws IOException {
     getGraphState().getWorkerCommunications().
-    removeVertexReq(vertexId);
+        removeVertexRequest(vertexId);
   }
 
   /**
@@ -115,7 +108,7 @@ public abstract class MutableVertex<I ex
   public void addEdgeRequest(I sourceVertexId, Edge<I, E> edge)
     throws IOException {
     getGraphState().getWorkerCommunications().
-    addEdgeReq(sourceVertexId, edge);
+        addEdgeRequest(sourceVertexId, edge);
   }
 
   /**
@@ -123,11 +116,11 @@ public abstract class MutableVertex<I ex
    * (processed just prior to the next superstep).
    *
    * @param sourceVertexId Source vertex id of edge
-   * @param destVertexId Destination vertex id of edge
+   * @param targetVertexId Destination vertex id of edge
    */
-  public void removeEdgeRequest(I sourceVertexId, I destVertexId)
+  public void removeEdgeRequest(I sourceVertexId, I targetVertexId)
     throws IOException {
     getGraphState().getWorkerCommunications().
-    removeEdgeReq(sourceVertexId, destVertexId);
+        removeEdgeRequest(sourceVertexId, targetVertexId);
   }
 }

Added: giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java?rev=1365352&view=auto
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java (added)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleMutableVertex.java Tue Jul 24 23:37:42 2012
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.graph;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Mutable vertex with no edge values.
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <M> Message data
+ */
+public abstract class SimpleMutableVertex<I extends WritableComparable,
+    V extends Writable, M extends Writable> extends MutableVertex<I, V,
+    NullWritable, M> {
+  /**
+   * Get a read-only view of the neighbors of this
+   * vertex, i.e. the target vertices of its out-edges.
+   *
+   * @return the neighbors (sort order determined by subclass implementation).
+   */
+  public abstract Iterable<I> getNeighbors();
+
+  @Override
+  public Iterable<Edge<I, NullWritable>> getEdges() {
+    return Iterables.transform(getNeighbors(), new Function<I, Edge<I,
+        NullWritable>>() {
+      public Edge<I, NullWritable> apply(I targetVertexId) {
+        return new Edge<I, NullWritable>(targetVertexId, NullWritable.get());
+      }
+    });
+  }
+
+  @Override
+  public NullWritable getEdgeValue(I targetVertexId) {
+    return NullWritable.get();
+  }
+
+  /**
+   * Add an edge for this vertex (happens immediately)
+   *
+   * @param targetVertexId target vertex
+   * @return Return true if succeeded, false otherwise
+   */
+  public abstract boolean addEdge(I targetVertexId);
+
+  @Override
+  public boolean addEdge(I targetVertexId, NullWritable value) {
+    return addEdge(targetVertexId);
+  }
+
+  /**
+   * Request to add an edge of a vertex in the graph
+   * (processed just prior to the next superstep)
+   *
+   * @param sourceVertexId Source vertex id of edge
+   */
+  public void addEdgeRequest(I sourceVertexId) throws IOException {
+    getGraphState().getWorkerCommunications().
+        addEdgeRequest(sourceVertexId, new Edge<I,
+            NullWritable>(sourceVertexId, NullWritable.get()));
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    I vertexId = BspUtils.<I>createVertexId(getConf());
+    vertexId.readFields(in);
+    V vertexValue = BspUtils.<V>createVertexValue(getConf());
+    vertexValue.readFields(in);
+
+    int numEdges = in.readInt();
+    Map<I, NullWritable> edges = new HashMap<I, NullWritable>(numEdges);
+    for (int i = 0; i < numEdges; ++i) {
+      I targetVertexId = BspUtils.<I>createVertexId(getConf());
+      targetVertexId.readFields(in);
+      edges.put(targetVertexId, NullWritable.get());
+    }
+
+    int numMessages = in.readInt();
+    List<M> messages = new ArrayList<M>(numMessages);
+    for (int i = 0; i < numMessages; ++i) {
+      M message = BspUtils.<M>createMessageValue(getConf());
+      message.readFields(in);
+      messages.add(message);
+    }
+    initialize(vertexId, vertexValue, edges, messages);
+
+    boolean halt = in.readBoolean();
+    if (halt) {
+      voteToHalt();
+    } else {
+      wakeUp();
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    getId().write(out);
+    getValue().write(out);
+
+    out.writeInt(getNumEdges());
+    for (I neighbor : getNeighbors()) {
+      neighbor.write(out);
+    }
+
+    out.writeInt(getNumMessages());
+    for (M message : getMessages()) {
+      message.write(out);
+    }
+
+    out.writeBoolean(isHalted());
+  }
+}

Added: giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleVertex.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleVertex.java?rev=1365352&view=auto
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleVertex.java (added)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/SimpleVertex.java Tue Jul 24 23:37:42 2012
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.graph;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * Vertex with no edge values.
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <M> Message data
+ */
+public abstract class SimpleVertex<I extends WritableComparable,
+    V extends Writable, M extends Writable> extends Vertex<I, V,
+    NullWritable, M> {
+  /**
+   * Get a read-only view of the neighbors of this
+   * vertex, i.e. the target vertices of its out-edges.
+   *
+   * @return the neighbors (sort order determined by subclass implementation).
+   */
+  public abstract Iterable<I> getNeighbors();
+
+  @Override
+  public Iterable<Edge<I, NullWritable>> getEdges() {
+    return Iterables.transform(getNeighbors(), new Function<I, Edge<I,
+        NullWritable>>() {
+
+      @Override
+      public Edge<I, NullWritable> apply(@Nullable I targetVertexId) {
+        return new Edge<I, NullWritable>(targetVertexId, NullWritable.get());
+      }
+    });
+  }
+
+  @Override
+  public NullWritable getEdgeValue(I targetVertexId) {
+    return NullWritable.get();
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    I vertexId = BspUtils.<I>createVertexId(getConf());
+    vertexId.readFields(in);
+    V vertexValue = BspUtils.<V>createVertexValue(getConf());
+    vertexValue.readFields(in);
+
+    int numEdges = in.readInt();
+    Map<I, NullWritable> edges = new HashMap<I, NullWritable>(numEdges);
+    for (int i = 0; i < numEdges; ++i) {
+      I targetVertexId = BspUtils.<I>createVertexId(getConf());
+      targetVertexId.readFields(in);
+      edges.put(targetVertexId, NullWritable.get());
+    }
+
+    int numMessages = in.readInt();
+    List<M> messages = new ArrayList<M>(numMessages);
+    for (int i = 0; i < numMessages; ++i) {
+      M message = BspUtils.<M>createMessageValue(getConf());
+      message.readFields(in);
+      messages.add(message);
+    }
+    initialize(vertexId, vertexValue, edges, messages);
+
+    boolean halt = in.readBoolean();
+    if (halt) {
+      voteToHalt();
+    } else {
+      wakeUp();
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    getId().write(out);
+    getValue().write(out);
+
+    out.writeInt(getNumEdges());
+    for (I neighbor : getNeighbors()) {
+      neighbor.write(out);
+    }
+
+    out.writeInt(getNumMessages());
+    for (M message : getMessages()) {
+      message.write(out);
+    }
+
+    out.writeBoolean(isHalted());
+  }
+}

Copied: giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java (from r1365343, giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java)
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java?p2=giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java&p1=giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java&r1=1365343&r2=1365352&rev=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java Tue Jul 24 23:37:42 2012
@@ -24,12 +24,13 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 
+import com.google.common.collect.Iterables;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -42,11 +43,15 @@ import java.util.Map;
  * @param <M> Message data
  */
 @SuppressWarnings("rawtypes")
-public abstract class BasicVertex<I extends WritableComparable,
+public abstract class Vertex<I extends WritableComparable,
     V extends Writable, E extends Writable, M extends Writable>
     implements AggregatorUsage, Writable, Configurable {
+  /** Vertex id. */
+  private I id;
+  /** Vertex value. */
+  private V value;
   /** If true, do not do anymore computation on this vertex. */
-  protected boolean halt = false;
+  private boolean halt;
   /** Global graph state **/
   private GraphState<I, V, E, M> graphState;
   /** Configuration */
@@ -57,22 +62,34 @@ public abstract class BasicVertex<I exte
    * This method must be called after instantiation of a vertex with BspUtils
    * unless deserialization from readFields() is called.
    *
-   * @param vertexId Will be the vertex id
-   * @param vertexValue Will be the vertex value
+   * @param id Will be the vertex id
+   * @param value Will be the vertex value
    * @param edges A map of destination edge ids to edge values (can be null)
    * @param messages Initial messages for this vertex (can be null)
    */
   public abstract void initialize(
-      I vertexId, V vertexValue, Map<I, E> edges, Iterable<M> messages);
+      I id, V value, Map<I, E> edges, Iterable<M> messages);
+
+  /**
+   * This method must be called once by the subclass's initialize() or by
+   * readFields() in order to set id and value.
+   *
+   * @param id Vertex id
+   * @param value Vertex value
+   */
+  public void initialize(I id, V value) {
+    this.id = id;
+    this.value = value;
+  }
 
   /**
    * Must be defined by user to do computation on a single Vertex.
    *
-   * @param msgIterator Iterator to the messages that were sent to this
-   *        vertex in the previous superstep
+   * @param messages Messages that were sent to this vertex in the previous
+   *                 superstep
    * @throws IOException
    */
-  public abstract void compute(Iterator<M> msgIterator) throws IOException;
+  public abstract void compute(Iterable<M> messages) throws IOException;
 
   /**
    * Retrieves the current superstep.
@@ -88,21 +105,27 @@ public abstract class BasicVertex<I exte
    *
    * @return My vertex id.
    */
-  public abstract I getVertexId();
+  public I getId() {
+    return id;
+  }
 
   /**
    * Get the vertex value (data stored with vertex)
    *
    * @return Vertex value
    */
-  public abstract V getVertexValue();
+  public V getValue() {
+    return value;
+  }
 
   /**
    * Set the vertex data (immediately visible in the computation)
    *
-   * @param vertexValue Vertex data to be set
+   * @param value Vertex data to be set
    */
-  public abstract void setVertexValue(V vertexValue);
+  public void setValue(V value) {
+    this.value = value;
+  }
 
   /**
    * Get the total (all workers) number of vertices that
@@ -110,8 +133,8 @@ public abstract class BasicVertex<I exte
    *
    * @return Total number of vertices (-1 if first superstep)
    */
-  public long getNumVertices() {
-    return getGraphState().getNumVertices();
+  public long getTotalNumVertices() {
+    return getGraphState().getTotalNumVertices();
   }
 
   /**
@@ -120,8 +143,8 @@ public abstract class BasicVertex<I exte
    *
    * @return Total number of edges (-1 if first superstep)
    */
-  public long getNumEdges() {
-    return getGraphState().getNumEdges();
+  public long getTotalNumEdges() {
+    return getGraphState().getTotalNumEdges();
   }
 
   /**
@@ -129,58 +152,74 @@ public abstract class BasicVertex<I exte
    *
    * @return the out edges (sort order determined by subclass implementation).
    */
-  public abstract Iterator<I> getOutEdgesIterator();
+  public abstract Iterable<Edge<I, E>> getEdges();
 
   /**
-   * Get the edge value associated with a target vertex id.
+   * Does an edge with the target vertex id exist?
    *
    * @param targetVertexId Target vertex id to check
-   *
-   * @return the value of the edge to targetVertexId (or null if there
-   *         is no edge to it)
+   * @return true if there is an edge to the target
    */
-  public abstract E getEdgeValue(I targetVertexId);
+  public boolean hasEdge(I targetVertexId) {
+    for (Edge<I, E> edge : getEdges()) {
+      if (edge.getTargetVertexId().equals(targetVertexId)) {
+        return true;
+      }
+    }
+    return false;
+  }
 
   /**
-   * Does an edge with the target vertex id exist?
+   * Get the edge value associated with a target vertex id.
    *
    * @param targetVertexId Target vertex id to check
-   * @return true if there is an edge to the target
+   *
+   * @return the value of the edge to targetVertexId (or null if there
+   *         is no edge to it)
    */
-  public abstract boolean hasEdge(I targetVertexId);
+  public E getEdgeValue(I targetVertexId) {
+    for (Edge<I, E> edge : getEdges()) {
+      if (edge.getTargetVertexId().equals(targetVertexId)) {
+        return edge.getValue();
+      }
+    }
+    return null;
+  }
 
   /**
    * Get the number of outgoing edges on this vertex.
    *
    * @return the total number of outbound edges from this vertex
    */
-  public abstract int getNumOutEdges();
+  public int getNumEdges() {
+    return Iterables.size(getEdges());
+  }
 
   /**
    * Send a message to a vertex id.  The message should not be mutated after
    * this method returns or else undefined results could occur.
    *
    * @param id Vertex id to send the message to
-   * @param msg Message data to send.  Note that after the message is sent,
+   * @param message Message data to send.  Note that after the message is sent,
    *        the user should not modify the object.
    */
-  public void sendMsg(I id, M msg) {
-    if (msg == null) {
+  public void sendMessage(I id, M message) {
+    if (message == null) {
       throw new IllegalArgumentException(
-          "sendMsg: Cannot send null message to " + id);
+          "sendMessage: Cannot send null message to " + id);
     }
     getGraphState().getWorkerCommunications().
-    sendMessageReq(id, msg);
+        sendMessageRequest(id, message);
   }
 
   /**
    * Send a message to all edges.
    *
-   * @param msg Message sent to all edges.
+   * @param message Message sent to all edges.
    */
-  public void sendMsgToAllEdges(M msg) {
-    for (Iterator<I> edges = getOutEdgesIterator(); edges.hasNext();) {
-      sendMsg(edges.next(), msg);
+  public void sendMessageToAllEdges(M message) {
+    for (Edge<I, E> edge : getEdges()) {
+      sendMessage(edge.getTargetVertexId(), message);
     }
   }
 
@@ -195,6 +234,13 @@ public abstract class BasicVertex<I exte
   }
 
   /**
+   * Re-activate vertex if halted.
+   */
+  public void wakeUp() {
+    halt = false;
+  }
+
+  /**
    * Is this vertex done?
    *
    * @return True if halted, false otherwise.
@@ -207,11 +253,19 @@ public abstract class BasicVertex<I exte
    *  Get the list of incoming messages from the previous superstep.  Same as
    *  the message iterator passed to compute().
    *
-   *  @return Iterator of messages.
+   *  @return Messages received.
    */
   public abstract Iterable<M> getMessages();
 
   /**
+   * Get the number of messages from the previous superstep.
+   * @return Number of messages received.
+   */
+  public int getNumMessages() {
+    return Iterables.size(getMessages());
+  }
+
+  /**
    * Copy the messages this vertex should process in the current superstep
    *
    * @param messages the messages sent to this vertex in the previous superstep
@@ -219,14 +273,8 @@ public abstract class BasicVertex<I exte
   abstract void putMessages(Iterable<M> messages);
 
   /**
-   * Get the number of incoming messages.
-   * @return the number of messages.
-   */
-  abstract int getNumMessages();
-
-  /**
    * Release unnecessary resources (will be called after vertex returns from
-   * {@link #compute()})
+   * {@link #compute(Iterable)})
    */
   abstract void releaseResources();
 
@@ -287,18 +335,8 @@ public abstract class BasicVertex<I exte
   }
 
   @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
   public void readFields(DataInput in) throws IOException {
-    I vertexId = BspUtils.<I>createVertexIndex(getConf());
+    I vertexId = BspUtils.<I>createVertexId(getConf());
     vertexId.readFields(in);
     V vertexValue = BspUtils.<V>createVertexValue(getConf());
     vertexValue.readFields(in);
@@ -306,7 +344,7 @@ public abstract class BasicVertex<I exte
     int numEdges = in.readInt();
     Map<I, E> edges = new HashMap<I, E>(numEdges);
     for (int i = 0; i < numEdges; ++i) {
-      I targetVertexId = BspUtils.<I>createVertexIndex(getConf());
+      I targetVertexId = BspUtils.<I>createVertexId(getConf());
       targetVertexId.readFields(in);
       E edgeValue = BspUtils.<E>createEdgeValue(getConf());
       edgeValue.readFields(in);
@@ -320,7 +358,6 @@ public abstract class BasicVertex<I exte
       message.readFields(in);
       messages.add(message);
     }
-
     initialize(vertexId, vertexValue, edges, messages);
 
     halt = in.readBoolean();
@@ -328,14 +365,13 @@ public abstract class BasicVertex<I exte
 
   @Override
   public void write(DataOutput out) throws IOException {
-    getVertexId().write(out);
-    getVertexValue().write(out);
+    getId().write(out);
+    getValue().write(out);
 
-    out.writeInt(getNumOutEdges());
-    for (Iterator<I> edges = getOutEdgesIterator(); edges.hasNext();) {
-      I targetVertexId = edges.next();
-      targetVertexId.write(out);
-      getEdgeValue(targetVertexId).write(out);
+    out.writeInt(getNumEdges());
+    for (Edge<I, E> edge : getEdges()) {
+      edge.getTargetVertexId().write(out);
+      edge.getValue().write(out);
     }
 
     out.writeInt(getNumMessages());
@@ -345,4 +381,20 @@ public abstract class BasicVertex<I exte
 
     out.writeBoolean(halt);
   }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public String toString() {
+    return "Vertex(id=" + getId() + ",value=" + getValue() +
+        ",#edges=" + getNumEdges() + ")";
+  }
 }

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/VertexChanges.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexChanges.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/VertexChanges.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/VertexChanges.java Tue Jul 24 23:37:42 2012
@@ -41,7 +41,7 @@ public interface VertexChanges<I extends
    *
    * @return List of vertices for this vertex index.
    */
-  List<BasicVertex<I, V, E, M>> getAddedVertexList();
+  List<Vertex<I, V, E, M>> getAddedVertexList();
 
   /**
    * Get the number of times this vertex was removed in the previous

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/VertexMutations.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexMutations.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/VertexMutations.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/VertexMutations.java Tue Jul 24 23:37:42 2012
@@ -46,8 +46,8 @@ public class VertexMutations<I extends W
     M extends Writable> implements VertexChanges<I, V, E, M>,
     Writable, Configurable {
   /** List of added vertices during the last superstep */
-  private final List<BasicVertex<I, V, E, M>> addedVertexList =
-      new ArrayList<BasicVertex<I, V, E, M>>();
+  private final List<Vertex<I, V, E, M>> addedVertexList =
+      new ArrayList<Vertex<I, V, E, M>>();
   /** Count of remove vertex requests */
   private int removedVertexCount = 0;
   /** List of added edges */
@@ -73,7 +73,7 @@ public class VertexMutations<I extends W
   }
 
   @Override
-  public List<BasicVertex<I, V, E, M>> getAddedVertexList() {
+  public List<Vertex<I, V, E, M>> getAddedVertexList() {
     return addedVertexList;
   }
 
@@ -85,14 +85,14 @@ public class VertexMutations<I extends W
 
     int addedVertexListSize = input.readInt();
     for (int i = 0; i < addedVertexListSize; ++i) {
-      BasicVertex<I, V, E, M> vertex = BspUtils.createVertex(conf);
+      Vertex<I, V, E, M> vertex = BspUtils.createVertex(conf);
       vertex.readFields(input);
       addedVertexList.add(vertex);
     }
     removedVertexCount = input.readInt();
     int addedEdgeListSize = input.readInt();
     for (int i = 0; i < addedEdgeListSize; ++i) {
-      I destVertex = BspUtils.<I>createVertexIndex(conf);
+      I destVertex = BspUtils.<I>createVertexId(conf);
       destVertex.readFields(input);
       E edgeValue = BspUtils.<E>createEdgeValue(conf);
       edgeValue.readFields(input);
@@ -100,7 +100,7 @@ public class VertexMutations<I extends W
     }
     int removedEdgeListSize = input.readInt();
     for (int i = 0; i < removedEdgeListSize; ++i) {
-      I removedEdge = BspUtils.<I>createVertexIndex(conf);
+      I removedEdge = BspUtils.<I>createVertexId(conf);
       removedEdge.readFields(input);
       removedEdgeList.add(removedEdge);
     }
@@ -109,14 +109,14 @@ public class VertexMutations<I extends W
   @Override
   public void write(DataOutput output) throws IOException {
     output.writeInt(addedVertexList.size());
-    for (BasicVertex<I, V, E, M> vertex : addedVertexList) {
+    for (Vertex<I, V, E, M> vertex : addedVertexList) {
       vertex.write(output);
     }
     output.writeInt(removedVertexCount);
     output.writeInt(addedEdgeList.size());
     for (Edge<I, E> edge : addedEdgeList) {
-      edge.getDestVertexId().write(output);
-      edge.getEdgeValue().write(output);
+      edge.getTargetVertexId().write(output);
+      edge.getValue().write(output);
     }
     output.writeInt(removedEdgeList.size());
     for (I removedEdge : removedEdgeList) {
@@ -129,7 +129,7 @@ public class VertexMutations<I extends W
    *
    * @param vertex Vertex to be added
    */
-  public void addVertex(BasicVertex<I, V, E, M> vertex) {
+  public void addVertex(Vertex<I, V, E, M> vertex) {
     addedVertexList.add(vertex);
   }
 

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/VertexReader.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexReader.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/VertexReader.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/VertexReader.java Tue Jul 24 23:37:42 2012
@@ -65,7 +65,7 @@ public interface VertexReader<I extends 
    * @throws IOException
    * @throws InterruptedException
    */
-  BasicVertex<I, V, E, M> getCurrentVertex()
+  Vertex<I, V, E, M> getCurrentVertex()
     throws IOException, InterruptedException;
 
   /**

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java Tue Jul 24 23:37:42 2012
@@ -18,13 +18,14 @@
 
 package org.apache.giraph.graph;
 
-import com.google.common.collect.Iterables;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.Iterables;
+
 import java.util.List;
 
 /**
@@ -48,13 +49,13 @@ public class VertexResolver<I extends Wr
   private GraphState<I, V, E, M> graphState;
 
   @Override
-  public BasicVertex<I, V, E, M> resolve(
+  public Vertex<I, V, E, M> resolve(
       I vertexId,
-      BasicVertex<I, V, E, M> vertex,
+      Vertex<I, V, E, M> vertex,
       VertexChanges<I, V, E, M> vertexChanges,
       Iterable<M> messages) {
     // Default algorithm:
-      // 1. If the vertex exists, first prune the edges
+    // 1. If the vertex exists, first prune the edges
     // 2. If vertex removal desired, remove the vertex.
     // 3. If creation of vertex desired, pick first vertex
     // 4. If vertex doesn't exist, but got messages, create
@@ -95,7 +96,7 @@ public class VertexResolver<I extends Wr
       if ((vertexChanges != null) &&
           (!vertexChanges.getAddedVertexList().isEmpty())) {
         LOG.warn("resolve: Tried to add a vertex with id = " +
-            vertex.getVertexId() + " when one already " +
+            vertex.getId() + " when one already " +
             "exists.  Ignoring the add vertex request.");
       }
     }
@@ -105,9 +106,8 @@ public class VertexResolver<I extends Wr
       MutableVertex<I, V, E, M> mutableVertex =
           (MutableVertex<I, V, E, M>) vertex;
       for (Edge<I, E> edge : vertexChanges.getAddedEdgeList()) {
-        edge.setConf(getConf());
-        mutableVertex.addEdge(edge.getDestVertexId(),
-            edge.getEdgeValue());
+        mutableVertex.addEdge(edge.getTargetVertexId(),
+            edge.getValue());
       }
     }
 
@@ -115,8 +115,8 @@ public class VertexResolver<I extends Wr
   }
 
   @Override
-  public BasicVertex<I, V, E, M> instantiateVertex() {
-    BasicVertex<I, V, E, M> vertex =
+  public Vertex<I, V, E, M> instantiateVertex() {
+    Vertex<I, V, E, M> vertex =
         BspUtils.<I, V, E, M>createVertex(getConf());
     vertex.setGraphState(graphState);
     return vertex;

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/VertexWriter.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexWriter.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/VertexWriter.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/VertexWriter.java Tue Jul 24 23:37:42 2012
@@ -50,7 +50,7 @@ public interface VertexWriter<I extends 
    * @throws IOException
    * @throws InterruptedException
    */
-  void writeVertex(BasicVertex<I, V, E, ?> vertex)
+  void writeVertex(Vertex<I, V, E, ?> vertex)
     throws IOException, InterruptedException;
 
   /**

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/WorkerContext.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/WorkerContext.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/WorkerContext.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/WorkerContext.java Tue Jul 24 23:37:42 2012
@@ -86,8 +86,8 @@ public abstract class WorkerContext impl
    *
    * @return Total number of vertices (-1 if first superstep)
    */
-  public long getNumVertices() {
-    return graphState.getNumVertices();
+  public long getTotalNumVertices() {
+    return graphState.getTotalNumVertices();
   }
 
   /**
@@ -96,8 +96,8 @@ public abstract class WorkerContext impl
    *
    * @return Total number of edges (-1 if first superstep)
    */
-  public long getNumEdges() {
-    return graphState.getNumEdges();
+  public long getTotalNumEdges() {
+    return graphState.getTotalNumEdges();
   }
 
   /**

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/partition/Partition.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/Partition.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/partition/Partition.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/partition/Partition.java Tue Jul 24 23:37:42 2012
@@ -18,19 +18,20 @@
 
 package org.apache.giraph.graph.partition;
 
+import org.apache.giraph.graph.BspUtils;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.collect.Maps;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.giraph.graph.BasicVertex;
-import org.apache.giraph.graph.BspUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-
 /**
  * A generic container that stores vertices.  Vertex ids will map to exactly
  * one partition.
@@ -47,20 +48,19 @@ public class Partition<I extends Writabl
   /** Configuration from the worker */
   private final Configuration conf;
   /** Partition id */
-  private final int partitionId;
+  private final int id;
   /** Vertex map for this range (keyed by index) */
-  private final Map<I, BasicVertex<I, V, E, M>> vertexMap =
-      new HashMap<I, BasicVertex<I, V, E, M>>();
+  private final Map<I, Vertex<I, V, E, M>> vertexMap = Maps.newHashMap();
 
   /**
    * Constructor.
    *
    * @param conf Configuration.
-   * @param partitionId Partition id.
+   * @param id Partition id.
    */
-  public Partition(Configuration conf, int partitionId) {
+  public Partition(Configuration conf, int id) {
     this.conf = conf;
-    this.partitionId = partitionId;
+    this.id = id;
   }
 
   /**
@@ -69,7 +69,7 @@ public class Partition<I extends Writabl
    * @param vertexIndex Vertex index to search for
    * @return Vertex if it exists, null otherwise
    */
-  public BasicVertex<I, V, E, M> getVertex(I vertexIndex) {
+  public Vertex<I, V, E, M> getVertex(I vertexIndex) {
     return vertexMap.get(vertexIndex);
   }
 
@@ -79,8 +79,8 @@ public class Partition<I extends Writabl
    * @param vertex Vertex to put in the Partition
    * @return old vertex value (i.e. null if none existed prior)
    */
-  public BasicVertex<I, V, E, M> putVertex(BasicVertex<I, V, E, M> vertex) {
-    return vertexMap.put(vertex.getVertexId(), vertex);
+  public Vertex<I, V, E, M> putVertex(Vertex<I, V, E, M> vertex) {
+    return vertexMap.put(vertex.getId(), vertex);
   }
 
   /**
@@ -89,7 +89,7 @@ public class Partition<I extends Writabl
    * @param vertexIndex Vertex index to remove
    * @return The removed vertex.
    */
-  public BasicVertex<I, V, E, M> removeVertex(I vertexIndex) {
+  public Vertex<I, V, E, M> removeVertex(I vertexIndex) {
     return vertexMap.remove(vertexIndex);
   }
 
@@ -98,7 +98,7 @@ public class Partition<I extends Writabl
    *
    * @return Collection of the vertices
    */
-  public Collection<BasicVertex<I, V, E , M>> getVertices() {
+  public Collection<Vertex<I, V, E , M>> getVertices() {
     return vertexMap.values();
   }
 
@@ -109,8 +109,8 @@ public class Partition<I extends Writabl
    */
   public long getEdgeCount() {
     long edges = 0;
-    for (BasicVertex<I, V, E, M> vertex : vertexMap.values()) {
-      edges += vertex.getNumOutEdges();
+    for (Vertex<I, V, E, M> vertex : vertexMap.values()) {
+      edges += vertex.getNumEdges();
     }
     return edges;
   }
@@ -118,15 +118,15 @@ public class Partition<I extends Writabl
   /**
    * Get the partition id.
    *
-   * @return Partition id of this partition.
+   * @return Id of this partition.
    */
-  public int getPartitionId() {
-    return partitionId;
+  public int getId() {
+    return id;
   }
 
   @Override
   public String toString() {
-    return "(id=" + getPartitionId() + ",V=" + vertexMap.size() +
+    return "(id=" + getId() + ",V=" + vertexMap.size() +
         ",E=" + getEdgeCount() + ")";
   }
 
@@ -134,11 +134,11 @@ public class Partition<I extends Writabl
   public void readFields(DataInput input) throws IOException {
     int vertices = input.readInt();
     for (int i = 0; i < vertices; ++i) {
-      BasicVertex<I, V, E, M> vertex =
+      Vertex<I, V, E, M> vertex =
         BspUtils.<I, V, E, M>createVertex(conf);
       vertex.readFields(input);
-      if (vertexMap.put(vertex.getVertexId(),
-          (BasicVertex<I, V, E, M>) vertex) != null) {
+      if (vertexMap.put(vertex.getId(),
+          (Vertex<I, V, E, M>) vertex) != null) {
         throw new IllegalStateException(
             "readFields: " + this +
             " already has same id " + vertex);
@@ -149,7 +149,7 @@ public class Partition<I extends Writabl
   @Override
   public void write(DataOutput output) throws IOException {
     output.writeInt(vertexMap.size());
-    for (BasicVertex vertex : vertexMap.values()) {
+    for (Vertex vertex : vertexMap.values()) {
       vertex.write(output);
     }
   }

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangePartitionOwner.java Tue Jul 24 23:37:42 2012
@@ -63,7 +63,7 @@ public class RangePartitionOwner<I exten
   @Override
   public void readFields(DataInput input) throws IOException {
     super.readFields(input);
-    maxIndex = BspUtils.<I>createVertexIndex(getConf());
+    maxIndex = BspUtils.<I>createVertexId(getConf());
     maxIndex.readFields(input);
   }
 

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/partition/RangeSplitHint.java Tue Jul 24 23:37:42 2012
@@ -48,7 +48,7 @@ public class RangeSplitHint<I extends Wr
 
   @Override
   public void readFields(DataInput input) throws IOException {
-    splitIndex = BspUtils.<I>createVertexIndex(conf);
+    splitIndex = BspUtils.<I>createVertexId(conf);
     splitIndex.readFields(input);
     preSplitVertexCount = input.readLong();
     postSplitVertexCount = input.readLong();

Modified: giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListTextVertexOutputFormat.java Tue Jul 24 23:37:42 2012
@@ -17,7 +17,8 @@
  */
 package org.apache.giraph.lib;
 
-import org.apache.giraph.graph.BasicVertex;
+import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexWriter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -26,7 +27,6 @@ import org.apache.hadoop.mapreduce.Recor
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
 import java.io.IOException;
-import java.util.Iterator;
 
 /**
  * OutputFormat to write out the graph nodes as text, value-separated (by
@@ -69,21 +69,20 @@ public class AdjacencyListTextVertexOutp
     }
 
     @Override
-    public void writeVertex(BasicVertex<I, V, E, ?> vertex) throws IOException,
+    public void writeVertex(Vertex<I, V, E, ?> vertex) throws IOException,
     InterruptedException {
       if (delimiter == null) {
         delimiter = getContext().getConfiguration()
             .get(LINE_TOKENIZE_VALUE, LINE_TOKENIZE_VALUE_DEFAULT);
       }
 
-      StringBuffer sb = new StringBuffer(vertex.getVertexId().toString());
+      StringBuffer sb = new StringBuffer(vertex.getId().toString());
       sb.append(delimiter);
-      sb.append(vertex.getVertexValue().toString());
+      sb.append(vertex.getValue());
 
-      for (Iterator<I> edges = vertex.getOutEdgesIterator(); edges.hasNext();) {
-        I edge = edges.next();
-        sb.append(delimiter).append(edge);
-        sb.append(delimiter).append(vertex.getEdgeValue(edge));
+      for (Edge<I, E> edge : vertex.getEdges()) {
+        sb.append(delimiter).append(edge.getTargetVertexId());
+        sb.append(delimiter).append(edge.getValue());
       }
 
       getRecordWriter().write(new Text(sb.toString()), null);

Modified: giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/lib/AdjacencyListVertexReader.java Tue Jul 24 23:37:42 2012
@@ -17,10 +17,9 @@
  */
 package org.apache.giraph.lib;
 
-import com.google.common.collect.Maps;
-import org.apache.giraph.graph.BasicVertex;
 import org.apache.giraph.graph.BspUtils;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.lib.TextVertexInputFormat.TextVertexReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.LongWritable;
@@ -29,6 +28,8 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.RecordReader;
 
+import com.google.common.collect.Maps;
+
 import java.io.IOException;
 import java.util.Map;
 
@@ -128,11 +129,11 @@ public abstract class AdjacencyListVerte
   }
 
   @Override
-  public BasicVertex<I, V, E, M> getCurrentVertex()
+  public Vertex<I, V, E, M> getCurrentVertex()
     throws IOException, InterruptedException {
     Configuration conf = getContext().getConfiguration();
     String line = getRecordReader().getCurrentValue().toString();
-    BasicVertex<I, V, E, M> vertex = BspUtils.createVertex(conf);
+    Vertex<I, V, E, M> vertex = BspUtils.createVertex(conf);
 
     if (sanitizer != null) {
       line = sanitizer.sanitize(line);
@@ -149,7 +150,7 @@ public abstract class AdjacencyListVerte
         "Line did not split correctly: " + line);
     }
 
-    I vertexId = BspUtils.<I>createVertexIndex(conf);
+    I vertexId = BspUtils.<I>createVertexId(conf);
     decodeId(values[0], vertexId);
 
     V value = BspUtils.<V>createVertexValue(conf);
@@ -160,7 +161,7 @@ public abstract class AdjacencyListVerte
     Edge<I, E> edge = new Edge<I, E>();
     while (i < values.length) {
       decodeEdge(values[i], values[i + 1], edge);
-      edges.put(edge.getDestVertexId(), edge.getEdgeValue());
+      edges.put(edge.getTargetVertexId(), edge.getValue());
       i += 2;
     }
     vertex.initialize(vertexId, value, edges, null);

Modified: giraph/trunk/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/lib/IdWithValueTextOutputFormat.java Tue Jul 24 23:37:42 2012
@@ -19,7 +19,7 @@
 package org.apache.giraph.lib;
 
 
-import org.apache.giraph.graph.BasicVertex;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexWriter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -74,7 +74,7 @@ public class IdWithValueTextOutputFormat
     }
 
     @Override
-    public void writeVertex(BasicVertex<I, V, E, ?> vertex) throws IOException,
+    public void writeVertex(Vertex<I, V, E, ?> vertex) throws IOException,
     InterruptedException {
       if (delimiter == null) {
         delimiter = getContext().getConfiguration()
@@ -87,11 +87,11 @@ public class IdWithValueTextOutputFormat
           .getBoolean(REVERSE_ID_AND_VALUE, REVERSE_ID_AND_VALUE_DEFAULT);
 
       if (reverseOutput) {
-        first = vertex.getVertexValue().toString();
-        second = vertex.getVertexId().toString();
+        first = vertex.getValue().toString();
+        second = vertex.getId().toString();
       } else {
-        first = vertex.getVertexId().toString();
-        second = vertex.getVertexValue().toString();
+        first = vertex.getId().toString();
+        second = vertex.getValue().toString();
       }
 
       Text line = new Text(first + delimiter + second);

Modified: giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java Tue Jul 24 23:37:42 2012
@@ -18,11 +18,8 @@
 
 package org.apache.giraph.lib;
 
-import com.google.common.collect.Maps;
-import net.iharder.Base64;
-import org.apache.giraph.graph.BasicVertex;
 import org.apache.giraph.graph.BspUtils;
-import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.LongWritable;
@@ -36,6 +33,10 @@ import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import com.google.common.collect.Maps;
+
+import net.iharder.Base64;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -84,10 +85,10 @@ public class JsonBase64VertexInputFormat
     }
 
     @Override
-    public BasicVertex<I, V, E, M> getCurrentVertex()
+    public Vertex<I, V, E, M> getCurrentVertex()
       throws IOException, InterruptedException {
       Configuration conf = getContext().getConfiguration();
-      BasicVertex<I, V, E, M> vertex = BspUtils.createVertex(conf);
+      Vertex<I, V, E, M> vertex = BspUtils.createVertex(conf);
 
       Text line = getRecordReader().getCurrentValue();
       JSONObject vertexObject;
@@ -105,7 +106,7 @@ public class JsonBase64VertexInputFormat
           vertexObject.getString(JsonBase64VertexFormat.VERTEX_ID_KEY));
         input = new DataInputStream(
           new ByteArrayInputStream(decodedWritable));
-        vertexId = BspUtils.<I>createVertexIndex(conf);
+        vertexId = BspUtils.<I>createVertexId(conf);
         vertexId.readFields(input);
       } catch (JSONException e) {
         throw new IllegalArgumentException(
@@ -141,10 +142,13 @@ public class JsonBase64VertexInputFormat
         }
         input = new DataInputStream(
             new ByteArrayInputStream(decodedWritable));
-        Edge<I, E> edge = new Edge<I, E>();
-        edge.setConf(getContext().getConfiguration());
-        edge.readFields(input);
-        edgeMap.put(edge.getDestVertexId(), edge.getEdgeValue());
+        I targetVertexId =
+            BspUtils.<I>createVertexId(getContext().getConfiguration());
+        targetVertexId.readFields(input);
+        E edgeValue =
+            BspUtils.<E>createEdgeValue(getContext().getConfiguration());
+        edgeValue.readFields(input);
+        edgeMap.put(targetVertexId, edgeValue);
       }
       vertex.initialize(vertexId, vertexValue, edgeMap, null);
       return vertex;

Modified: giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java Tue Jul 24 23:37:42 2012
@@ -18,9 +18,8 @@
 
 package org.apache.giraph.lib;
 
-import net.iharder.Base64;
-import org.apache.giraph.graph.BasicVertex;
 import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexWriter;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
@@ -31,11 +30,12 @@ import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import net.iharder.Base64;
+
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.util.Iterator;
 
 /**
  * Simple way to represent the structure of the graph with a JSON object.
@@ -72,13 +72,13 @@ public class JsonBase64VertexOutputForma
     }
 
     @Override
-    public void writeVertex(BasicVertex<I, V, E, ?> vertex)
+    public void writeVertex(Vertex<I, V, E, ?> vertex)
       throws IOException, InterruptedException {
       ByteArrayOutputStream outputStream =
           new ByteArrayOutputStream();
       DataOutput output = new DataOutputStream(outputStream);
       JSONObject vertexObject = new JSONObject();
-      vertex.getVertexId().write(output);
+      vertex.getId().write(output);
       try {
         vertexObject.put(
           JsonBase64VertexFormat.VERTEX_ID_KEY,
@@ -88,7 +88,7 @@ public class JsonBase64VertexOutputForma
             "writerVertex: Failed to insert vertex id", e);
       }
       outputStream.reset();
-      vertex.getVertexValue().write(output);
+      vertex.getValue().write(output);
       try {
         vertexObject.put(
           JsonBase64VertexFormat.VERTEX_VALUE_KEY,
@@ -98,14 +98,10 @@ public class JsonBase64VertexOutputForma
             "writerVertex: Failed to insert vertex value", e);
       }
       JSONArray edgeArray = new JSONArray();
-      for (Iterator<I> edges = vertex.getOutEdgesIterator();
-           edges.hasNext();) {
-        I targetVertexId = edges.next();
-        Edge<I, E> edge = new Edge<I, E>(
-            targetVertexId, vertex.getEdgeValue(targetVertexId));
-        edge.setConf(getContext().getConfiguration());
+      for (Edge<I, E> edge : vertex.getEdges()) {
         outputStream.reset();
-        edge.write(output);
+        edge.getTargetVertexId().write(output);
+        edge.getValue().write(output);
         edgeArray.put(Base64.encodeBytes(outputStream.toByteArray()));
       }
       try {

Modified: giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexInputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexInputFormat.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexInputFormat.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexInputFormat.java Tue Jul 24 23:37:42 2012
@@ -17,9 +17,9 @@
  */
 package org.apache.giraph.lib;
 
+import org.apache.giraph.graph.BspUtils;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexReader;
-import org.json.JSONArray;
-import org.json.JSONException;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -27,9 +27,11 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.json.JSONArray;
+import org.json.JSONException;
+
 import com.google.common.collect.Maps;
-import org.apache.giraph.graph.BasicVertex;
-import org.apache.giraph.graph.BspUtils;
+
 import java.io.IOException;
 import java.util.Map;
 
@@ -77,12 +79,11 @@ public class JsonLongDoubleFloatDoubleVe
     }
 
     @Override
-    public BasicVertex<LongWritable, DoubleWritable, FloatWritable,
-      DoubleWritable> getCurrentVertex()
+    public Vertex<LongWritable, DoubleWritable, FloatWritable,
+          DoubleWritable> getCurrentVertex()
       throws IOException, InterruptedException {
-      BasicVertex<LongWritable, DoubleWritable, FloatWritable,
-      DoubleWritable> vertex =
-        BspUtils.<LongWritable, DoubleWritable, FloatWritable,
+      Vertex<LongWritable, DoubleWritable, FloatWritable, DoubleWritable>
+          vertex = BspUtils.<LongWritable, DoubleWritable, FloatWritable,
           DoubleWritable>createVertex(getContext().getConfiguration());
 
       Text line = getRecordReader().getCurrentValue();

Modified: giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexOutputFormat.java?rev=1365352&r1=1365351&r2=1365352&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexOutputFormat.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/lib/JsonLongDoubleFloatDoubleVertexOutputFormat.java Tue Jul 24 23:37:42 2012
@@ -18,7 +18,8 @@
 
 package org.apache.giraph.lib;
 
-import org.apache.giraph.graph.BasicVertex;
+import org.apache.giraph.graph.Edge;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexWriter;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -30,7 +31,6 @@ import org.json.JSONArray;
 import org.json.JSONException;
 
 import java.io.IOException;
-import java.util.Iterator;
 
 /**
  * VertexOutputFormat that supports JSON encoded vertices featuring
@@ -66,19 +66,17 @@ public class JsonLongDoubleFloatDoubleVe
     }
 
     @Override
-    public void writeVertex(BasicVertex<LongWritable, DoubleWritable,
-      FloatWritable, ?> vertex) throws IOException, InterruptedException {
+    public void writeVertex(Vertex<LongWritable, DoubleWritable,
+          FloatWritable, ?> vertex) throws IOException, InterruptedException {
       JSONArray jsonVertex = new JSONArray();
       try {
-        jsonVertex.put(vertex.getVertexId().get());
-        jsonVertex.put(vertex.getVertexValue().get());
+        jsonVertex.put(vertex.getId().get());
+        jsonVertex.put(vertex.getValue().get());
         JSONArray jsonEdgeArray = new JSONArray();
-        for (Iterator<LongWritable> edges = vertex.getOutEdgesIterator();
-             edges.hasNext();) {
-          LongWritable targetVertexId = edges.next();
+        for (Edge<LongWritable, FloatWritable> edge : vertex.getEdges()) {
           JSONArray jsonEdge = new JSONArray();
-          jsonEdge.put(targetVertexId.get());
-          jsonEdge.put(vertex.getEdgeValue(targetVertexId).get());
+          jsonEdge.put(edge.getTargetVertexId().get());
+          jsonEdge.put(edge.getValue().get());
           jsonEdgeArray.put(jsonEdge);
         }
         jsonVertex.put(jsonEdgeArray);



Mime
View raw message