You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ac...@apache.org on 2011/09/14 06:13:55 UTC

svn commit: r1170431 - in /incubator/giraph/trunk: ./ src/main/java/org/apache/giraph/benchmark/ src/main/java/org/apache/giraph/examples/ src/main/java/org/apache/giraph/graph/ src/main/java/org/apache/giraph/lib/

Author: aching
Date: Wed Sep 14 04:13:54 2011
New Revision: 1170431

URL: http://svn.apache.org/viewvc?rev=1170431&view=rev
Log:
GIRAPH-31: Hide the SortedMap<I, Edge<I,E>> in Vertex from client
visibility (impl. detail), replace with appropriate accessor
methods. jake.mannix via aching.


Modified:
    incubator/giraph/trunk/CHANGELOG
    incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexRange.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java
    incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java

Modified: incubator/giraph/trunk/CHANGELOG
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/CHANGELOG?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/CHANGELOG (original)
+++ incubator/giraph/trunk/CHANGELOG Wed Sep 14 04:13:54 2011
@@ -2,6 +2,10 @@ Giraph Change Log
 
 Release 0.70.0 - unreleased
 
+  GIRAPH-31: Hide the SortedMap<I, Edge<I,E>> in Vertex from client
+  visibility (impl. detail), replace with appropriate accessor
+  methods. jake.mannix via aching.
+
   GIRAPH-30: NPE in ZooKeeperManager if base directory cannot be
   created. apurtell via aching.
 

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java Wed Sep 14 04:13:54 2011
@@ -18,14 +18,6 @@
 
 package org.apache.giraph.benchmark;
 
-import java.util.Iterator;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.HelpFormatter;
@@ -33,6 +25,13 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
 import org.apache.giraph.graph.GiraphJob;
 import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.util.Iterator;
 
 /**
  * Benchmark based on the basic Pregel PageRank implementation.
@@ -72,7 +71,7 @@ public class PageRankBenchmark extends
         }
 
         if (getSuperstep() < getConf().getInt(SUPERSTEP_COUNT, -1)) {
-            long edges = getOutEdgeMap().size();
+            long edges = getNumOutEdges();
             sendMsgToAllEdges(
                 new DoubleWritable(getVertexValue().get() / edges));
         } else {

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/benchmark/PseudoRandomVertexInputFormat.java Wed Sep 14 04:13:54 2011
@@ -18,11 +18,10 @@
 
 package org.apache.giraph.benchmark;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
+import org.apache.giraph.bsp.BspInputSplit;
+import org.apache.giraph.graph.MutableVertex;
+import org.apache.giraph.graph.VertexInputFormat;
+import org.apache.giraph.graph.VertexReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -31,11 +30,10 @@ import org.apache.hadoop.mapreduce.JobCo
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
-import org.apache.giraph.bsp.BspInputSplit;
-import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.MutableVertex;
-import org.apache.giraph.graph.VertexInputFormat;
-import org.apache.giraph.graph.VertexReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
 
 /**
  * This VertexInputFormat is meant for large scale testing.  It allows the user
@@ -74,10 +72,6 @@ public class PseudoRandomVertexInputForm
     /**
      * Used by {@link PseudoRandomVertexInputFormat} to read
      * pseudo-randomly generated data
-     *
-     * @param <I> Vertex index value
-     * @param <V> Vertex value
-     * @param <E> Edge value
      */
     private static class PseudoRandomVertexReader implements
             VertexReader<LongWritable, DoubleWritable, DoubleWritable> {
@@ -157,11 +151,8 @@ public class PseudoRandomVertexInputForm
                     destVertexId =
                         new LongWritable(Math.abs(rand.nextLong()) %
                                          aggregateVertices);
-                } while (vertex.getOutEdgeMap().containsKey(destVertexId));
-                Edge<LongWritable, DoubleWritable> edge =
-                    new Edge<LongWritable, DoubleWritable>(
-                        destVertexId, new DoubleWritable(rand.nextDouble()));
-                vertex.addEdge(edge);
+                } while (vertex.hasEdge(destVertexId));
+                vertex.addEdge(destVertexId, new DoubleWritable(rand.nextDouble()));
             }
 
             ++verticesRead;
@@ -169,7 +160,7 @@ public class PseudoRandomVertexInputForm
                 LOG.debug("next: Return vertexId=" +
                           vertex.getVertexId().get() +
                           ", vertexValue=" + vertex.getVertexValue() +
-                          ", edgeMap=" + vertex.getOutEdgeMap());
+                          ", edgeMap=" + vertex.iterator());
             }
             return true;
         }

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleCheckpointVertex.java Wed Sep 14 04:13:54 2011
@@ -18,13 +18,8 @@
 
 package org.apache.giraph.examples;
 
-import java.util.Iterator;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.PosixParser;
+import org.apache.commons.cli.*;
+import org.apache.giraph.graph.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.FloatWritable;
@@ -34,11 +29,7 @@ import org.apache.hadoop.mapreduce.lib.o
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import org.apache.giraph.graph.GiraphJob;
-import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexInputFormat;
-import org.apache.giraph.graph.VertexOutputFormat;
+import java.util.Iterator;
 
 /**
  * An example that simply uses its id, value, and edges to compute new data
@@ -128,17 +119,17 @@ public class SimpleCheckpointVertex exte
         System.out.println("compute: vertex " + getVertexId() +
                            " has value " + getVertexValue() +
                            " on superstep " + getSuperstep());
-        for (Edge<LongWritable, FloatWritable> edge : getOutEdgeMap().values()) {
-            float edgeValue = edge.getEdgeValue().get();
+        for (LongWritable targetVertexId : this) {
+            FloatWritable edgeValue = getEdgeValue(targetVertexId);
             System.out.println("compute: vertex " + getVertexId() +
                                " sending edgeValue " + edgeValue +
                                " vertexValue " + vertexValue +
-                               " total " + (edgeValue + (float) vertexValue) +
-                               " to vertex " + edge.getDestVertexId() +
+                               " total " + (edgeValue.get() + (float) vertexValue) +
+                               " to vertex " + targetVertexId +
                                " on superstep " + getSuperstep());
-            edge.getEdgeValue().set(edgeValue + (float) vertexValue);
-            sendMsg(edge.getDestVertexId(),
-                    new FloatWritable(edgeValue));
+            edgeValue.set(edgeValue.get() + (float) vertexValue);
+            addEdge(targetVertexId, edgeValue);
+            sendMsg(targetVertexId, edgeValue);
         }
     }
 

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleFailVertex.java Wed Sep 14 04:13:54 2011
@@ -18,13 +18,12 @@
 
 package org.apache.giraph.examples;
 
-import java.util.Iterator;
-
+import org.apache.giraph.graph.Vertex;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
 
-import org.apache.giraph.graph.Vertex;
+import java.util.Iterator;
 
 /**
  * Vertex to allow unit testing of failure detection
@@ -56,7 +55,7 @@ public class SimpleFailVertex extends
                         return;
                     }
                 }
-                long edges = getOutEdgeMap().size();
+                long edges = getNumOutEdges();
                 sendMsgToAllEdges(
                     new DoubleWritable(getVertexValue().get() / edges));
             } else {

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimplePageRankVertex.java Wed Sep 14 04:13:54 2011
@@ -18,9 +18,9 @@
 
 package org.apache.giraph.examples;
 
-import java.io.IOException;
-import java.util.Iterator;
-
+import org.apache.giraph.graph.*;
+import org.apache.giraph.lib.TextVertexOutputFormat;
+import org.apache.giraph.lib.TextVertexOutputFormat.TextVertexWriter;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -28,17 +28,10 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
 import org.apache.log4j.Logger;
 
-import org.apache.giraph.graph.BasicVertex;
-import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.MutableVertex;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexReader;
-import org.apache.giraph.graph.VertexWriter;
-import org.apache.giraph.lib.TextVertexOutputFormat;
-import org.apache.giraph.lib.TextVertexOutputFormat.TextVertexWriter;
+import java.io.IOException;
+import java.util.Iterator;
 
 /**
  * Demonstrates the basic Pregel PageRank implementation.
@@ -123,7 +116,7 @@ public class SimplePageRankVertex extend
         }
 
         if (getSuperstep() < 30) {
-            long edges = getOutEdgeMap().size();
+            long edges = getNumOutEdges();
             sendMsgToAllEdges(
                 new DoubleWritable(getVertexValue().get() / edges));
         } else {
@@ -154,9 +147,8 @@ public class SimplePageRankVertex extend
                 (inputSplit.getNumSplits() * totalRecords);
             float edgeValue = vertex.getVertexId().get() * 100f;
             // Adds an edge to the neighbor vertex
-            vertex.addEdge(new Edge<LongWritable, FloatWritable>(
-                    new LongWritable(destVertexId),
-                    new FloatWritable(edgeValue)));
+            vertex.addEdge(new LongWritable(destVertexId),
+                    new FloatWritable(edgeValue));
             ++recordsRead;
             LOG.info("next: Return vertexId=" + vertex.getVertexId().get() +
                 ", vertexValue=" + vertex.getVertexValue() +

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleShortestPathsVertex.java Wed Sep 14 04:13:54 2011
@@ -18,9 +18,11 @@
 
 package org.apache.giraph.examples;
 
-import java.io.IOException;
-import java.util.Iterator;
-
+import org.apache.giraph.graph.*;
+import org.apache.giraph.lib.TextVertexInputFormat;
+import org.apache.giraph.lib.TextVertexInputFormat.TextVertexReader;
+import org.apache.giraph.lib.TextVertexOutputFormat;
+import org.apache.giraph.lib.TextVertexOutputFormat.TextVertexWriter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DoubleWritable;
@@ -36,21 +38,12 @@ import org.apache.hadoop.mapreduce.lib.o
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Logger;
-
-import org.apache.giraph.graph.BasicVertex;
-import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.GiraphJob;
-import org.apache.giraph.graph.MutableVertex;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexReader;
-import org.apache.giraph.graph.VertexWriter;
-import org.apache.giraph.lib.TextVertexInputFormat;
-import org.apache.giraph.lib.TextVertexInputFormat.TextVertexReader;
-import org.apache.giraph.lib.TextVertexOutputFormat;
-import org.apache.giraph.lib.TextVertexOutputFormat.TextVertexWriter;
 import org.json.JSONArray;
 import org.json.JSONException;
 
+import java.io.IOException;
+import java.util.Iterator;
+
 /**
  * Demonstrates the basic Pregel shortest paths implementation.
  */
@@ -93,16 +86,15 @@ public class SimpleShortestPathsVertex e
         }
         if (minDist < getVertexValue().get()) {
             setVertexValue(new DoubleWritable(minDist));
-            for (Edge<LongWritable, FloatWritable> edge :
-                    getOutEdgeMap().values()) {
+            for (LongWritable targetVertexId : this) {
+                FloatWritable edgeValue = getEdgeValue(targetVertexId);
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("Vertex " + getVertexId() + " sent to " +
-                              edge.getDestVertexId() + " = " +
-                              (minDist + edge.getEdgeValue().get()));
+                              targetVertexId + " = " +
+                              (minDist + edgeValue.get()));
                 }
-                sendMsg(edge.getDestVertexId(),
-                        new DoubleWritable(minDist +
-                                           edge.getEdgeValue().get()));
+                sendMsg(targetVertexId,
+                        new DoubleWritable(minDist + edgeValue.get()));
             }
         }
         voteToHalt();
@@ -160,11 +152,8 @@ public class SimpleShortestPathsVertex e
                 JSONArray jsonEdgeArray = jsonVertex.getJSONArray(2);
                 for (int i = 0; i < jsonEdgeArray.length(); ++i) {
                     JSONArray jsonEdge = jsonEdgeArray.getJSONArray(i);
-                    Edge<LongWritable, FloatWritable> edge =
-                        new Edge<LongWritable, FloatWritable>(
-                            new LongWritable(jsonEdge.getLong(0)),
+                    vertex.addEdge(new LongWritable(jsonEdge.getLong(0)),
                             new FloatWritable((float) jsonEdge.getDouble(1)));
-                    vertex.addEdge(edge);
                 }
             } catch (JSONException e) {
                 throw new IllegalArgumentException(
@@ -210,11 +199,10 @@ public class SimpleShortestPathsVertex e
                 jsonVertex.put(vertex.getVertexId().get());
                 jsonVertex.put(vertex.getVertexValue().get());
                 JSONArray jsonEdgeArray = new JSONArray();
-                for (Edge<LongWritable, FloatWritable> edge :
-                        vertex.getOutEdgeMap().values()) {
+                for (LongWritable targetVertexId : vertex) {
                     JSONArray jsonEdge = new JSONArray();
-                    jsonEdge.put(edge.getDestVertexId().get());
-                    jsonEdge.put(edge.getEdgeValue().get());
+                    jsonEdge.put(targetVertexId.get());
+                    jsonEdge.put(vertex.getEdgeValue(targetVertexId).get());
                     jsonEdgeArray.put(jsonEdge);
                 }
                 jsonVertex.put(jsonEdgeArray);

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/examples/SimpleSuperstepVertex.java Wed Sep 14 04:13:54 2011
@@ -18,9 +18,13 @@
 
 package org.apache.giraph.examples;
 
-import java.io.IOException;
-import java.util.Iterator;
-
+import org.apache.giraph.graph.BasicVertex;
+import org.apache.giraph.graph.MutableVertex;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.graph.VertexReader;
+import org.apache.giraph.graph.VertexWriter;
+import org.apache.giraph.lib.TextVertexOutputFormat;
+import org.apache.giraph.lib.TextVertexOutputFormat.TextVertexWriter;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -30,14 +34,8 @@ import org.apache.hadoop.mapreduce.Recor
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
-import org.apache.giraph.graph.BasicVertex;
-import org.apache.giraph.graph.Edge;
-import org.apache.giraph.graph.MutableVertex;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexReader;
-import org.apache.giraph.graph.VertexWriter;
-import org.apache.giraph.lib.TextVertexOutputFormat;
-import org.apache.giraph.lib.TextVertexOutputFormat.TextVertexWriter;
+import java.io.IOException;
+import java.util.Iterator;
 
 /**
  * Just a simple Vertex compute implementation that executes 3 supersteps, then
@@ -75,9 +73,8 @@ public class SimpleSuperstepVertex exten
                 (inputSplit.getNumSplits() * totalRecords);
             float edgeValue = vertex.getVertexId().get() * 100f;
             // Adds an edge to the neighbor vertex
-            vertex.addEdge(new Edge<LongWritable, FloatWritable>(
-                    new LongWritable(destVertexId),
-                    new FloatWritable(edgeValue)));
+            vertex.addEdge(new LongWritable(destVertexId),
+                    new FloatWritable(edgeValue));
             ++recordsRead;
             LOG.info("next: Return vertexId=" + vertex.getVertexId().get() +
                 ", vertexValue=" + vertex.getVertexValue() +

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BasicVertex.java Wed Sep 14 04:13:54 2011
@@ -25,7 +25,6 @@ import org.apache.hadoop.mapreduce.Mappe
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
-import java.util.SortedMap;
 
 /**
  * Basic interface for writing a BSP application for computation.
@@ -38,7 +37,7 @@ import java.util.SortedMap;
 @SuppressWarnings("rawtypes")
 public abstract class BasicVertex<I extends WritableComparable,
         V extends Writable, E extends Writable, M extends Writable>
-        implements AggregatorUsage {
+        implements AggregatorUsage, Iterable<I> {
     /** Global graph state **/
     private GraphState<I,V,E,M> graphState;
 
@@ -128,12 +127,37 @@ public abstract class BasicVertex<I exte
     }
 
     /**
-     * Every vertex has edges to other vertices.  Get a handle to the outward
-     * edge set.
+     * Get a read-only view of the out-edges of this vertex.
      *
-     * @return Map of the destination vertex index to the {@link Edge}
+     * @return the out edges (sort order determined by subclass implementation).
      */
-    public abstract SortedMap<I, Edge<I, E>> getOutEdgeMap();
+    @Override
+    public abstract Iterator<I> iterator();
+
+    /**
+     * Get the edge value associated with a target vertex id.
+     *
+     * @param targetVertexId Target vertex id to check
+     *
+     * @return the value of the edge to targetVertexId (or null if there
+     *         is no edge to it)
+     */
+    public abstract E getEdgeValue(I targetVertexId);
+
+    /**
+     * Does an edge with the target vertex id exist?
+     *
+     * @param targetVertexId Target vertex id to check
+     * @return true if there is an edge to the target
+     */
+    public abstract boolean hasEdge(I targetVertexId);
+
+    /**
+     * Get the number of outgoing edges on this vertex.
+     *
+     * @return the total number of outbound edges from this vertex
+     */
+    public abstract int getNumOutEdges();
 
     /**
      * Send a message to a vertex id.

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/BspServiceWorker.java Wed Sep 14 04:13:54 2011
@@ -564,7 +564,7 @@ public class BspServiceWorker<
             long vertexRangeEdgeCount = 0;
             for (BasicVertex<I, V, E, M> vertex :
                 entry.getValue().getVertexMap().values()) {
-                vertexRangeEdgeCount += vertex.getOutEdgeMap().size();
+                vertexRangeEdgeCount += vertex.getNumOutEdges();
             }
             statList.add(Long.valueOf(entry.getValue().getVertexMap().size()));
             statList.add(Long.valueOf(vertexRangeEdgeCount));
@@ -1108,7 +1108,7 @@ public class BspServiceWorker<
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("storeCheckpoint: Wrote vertex id = " +
                               vertex.getVertexId() + " with " +
-                              vertex.getOutEdgeMap().size() + " edges and " +
+                              vertex.getNumOutEdges() + " edges and " +
                               vertex.getMsgList().size() + " messages (" +
                               vertexByteStream.size() + " total bytes)");
                 }
@@ -1123,7 +1123,7 @@ public class BspServiceWorker<
             long edgeCount = 0;
             for (BasicVertex<I, V, E, M> vertex :
                     entry.getValue().getVertexMap().values()) {
-                edgeCount += vertex.getOutEdgeMap().size();
+                edgeCount += vertex.getNumOutEdges();
             }
             metadataOutput.writeLong(edgeCount);
             entry.getKey().write(metadataOutput);

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java Wed Sep 14 04:13:54 2011
@@ -627,7 +627,7 @@ public class GraphMapper<I extends Writa
                         ++workerFinishedVertices;
                     }
                     ++workerVertices;
-                    workerEdges += vertex.getOutEdgeMap().size();
+                    workerEdges += vertex.getNumOutEdges();
                 }
             }
 

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/MutableVertex.java Wed Sep 14 04:13:54 2011
@@ -18,11 +18,11 @@
 
 package org.apache.giraph.graph;
 
-import java.io.IOException;
-
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
+import java.io.IOException;
+
 /**
  * Interface used by VertexReader to set the properties of a new vertex
  * or mutate the graph.
@@ -41,10 +41,20 @@ public abstract class MutableVertex<I ex
     /**
      * Add an edge for this vertex (happens immediately)
      *
-     * @param edge Edge to be added
+     * @param targetVertexId target vertex
+     * @param edgeValue value of the edge
      * @return Return true if succeeded, false otherwise
      */
-    public abstract boolean addEdge(Edge<I, E> edge);
+    public abstract boolean addEdge(I targetVertexId, E edgeValue);
+
+    /**
+     * Removes an edge for this vertex (happens immediately).
+     *
+     * @param targetVertexId the target vertex id of the edge to be removed.
+     * @return the value of the edge which was removed (or null if no
+     *         edge existed to targetVertexId)
+     */
+    public abstract E removeEdge(I targetVertexId);
 
     /**
      * Create a vertex for use in addVertexRequest().  Still need to get the

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/Vertex.java Wed Sep 14 04:13:54 2011
@@ -25,14 +25,15 @@ import org.apache.log4j.Logger;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import java.util.*;
 
 /**
- * User applications should all subclass {@link Vertex}.  Package access
- * should prevent users from accessing internal methods.
+ * User applications often subclass {@link Vertex}, which stores the outbound
+ * edges in SortedMap, for both random-access and range operations.
+ * User applications which need to implement their own
+ * in-memory data structures should subclass {@link MutableVertex}.
+ *
+ * Package access will prevent users from accessing internal methods.
  *
  * @param <I> Vertex index value
  * @param <V> Vertex value
@@ -50,7 +51,7 @@ public abstract class Vertex<I extends W
     /** Vertex value */
     private V vertexValue = null;
     /** Map of destination vertices and their edge values */
-    private final SortedMap<I, Edge<I, E>> destEdgeMap =
+    protected final SortedMap<I, Edge<I, E>> destEdgeMap =
         new TreeMap<I, Edge<I, E>>();
     /** If true, do not do anymore computation on this vertex. */
     boolean halt = false;
@@ -79,13 +80,14 @@ public abstract class Vertex<I extends W
     }
 
     @Override
-    public final boolean addEdge(Edge<I, E> edge) {
-        edge.setConf(getContext().getConfiguration());
-        if (destEdgeMap.put(edge.getDestVertexId(), edge) != null) {
+    public final boolean addEdge(I targetVertexId, E edgeValue) {
+        if (destEdgeMap.put(
+                targetVertexId,
+                new Edge<I, E>(targetVertexId, edgeValue)) != null) {
             if (LOG.isDebugEnabled()) {
                 LOG.debug("addEdge: Vertex=" + vertexId +
                           ": already added an edge value for dest vertex id " +
-                          edge.getDestVertexId());
+                          targetVertexId);
             }
             return false;
         } else {
@@ -114,8 +116,40 @@ public abstract class Vertex<I extends W
     }
 
     @Override
-    public final SortedMap<I, Edge<I, E>> getOutEdgeMap() {
-        return destEdgeMap;
+    public E getEdgeValue(I targetVertexId) {
+        Edge<I, E> edge = destEdgeMap.get(targetVertexId);
+        return edge != null ? edge.getEdgeValue() : null;
+    }
+
+    @Override
+    public boolean hasEdge(I targetVertexId) {
+        return destEdgeMap.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 Iterator<I> iterator() {
+        return destEdgeMap.keySet().iterator();
+    }
+
+    @Override
+    public int getNumOutEdges() {
+        return destEdgeMap.size();
+    }
+
+    @Override
+    public E removeEdge(I targetVertexId) {
+        Edge<I, E> edge = destEdgeMap.remove(targetVertexId);
+        if(edge != null) {
+            return edge.getEdgeValue();
+        } else {
+            return null;
+        }
     }
 
     @Override
@@ -155,7 +189,7 @@ public abstract class Vertex<I extends W
             Edge<I, E> edge = new Edge<I, E>();
             edge.setConf(getContext().getConfiguration());
             edge.readFields(in);
-            addEdge(edge);
+            addEdge(edge.getDestVertexId(), edge.getEdgeValue());
         }
         long msgListSize = in.readLong();
         for (long i = 0; i < msgListSize; ++i) {
@@ -193,7 +227,7 @@ public abstract class Vertex<I extends W
     @Override
     public String toString() {
         return "Vertex(id=" + getVertexId() + ",value=" + getVertexValue() +
-            ",#edges=" + getOutEdgeMap().size() + ")";
+            ",#edges=" + destEdgeMap.size() + ")";
     }
 }
 

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexRange.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexRange.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexRange.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexRange.java Wed Sep 14 04:13:54 2011
@@ -18,25 +18,17 @@
 
 package org.apache.giraph.graph;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
 import net.iharder.Base64;
-
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.log4j.Logger;
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import java.io.*;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 /**
  * Defines a vertex index range and assigns responsibility to a particular
  * host and port.
@@ -305,7 +297,7 @@ public class VertexRange<I extends Writa
     public long getEdgeCount() {
         long edgeCount = 0;
         for (BasicVertex<I, V, E, M> vertex : vertexMap.values()) {
-            edgeCount += vertex.getOutEdgeMap().size();
+            edgeCount += vertex.getNumOutEdges();
         }
         return edgeCount;
     }

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/graph/VertexResolver.java Wed Sep 14 04:13:54 2011
@@ -62,8 +62,10 @@ public class VertexResolver<I extends Wr
             if (vertexChanges != null) {
                 List<I> removedEdgeList = vertexChanges.getRemovedEdgeList();
                 for (I removedDestVertex : removedEdgeList) {
-                    if (vertex.getOutEdgeMap().remove(removedDestVertex) ==
-                            null) {
+                    E removeEdge =
+                        ((MutableVertex<I, V, E, M>) vertex).removeEdge(
+                            removedDestVertex);
+                    if (removeEdge == null) {
                         LOG.warn("resolve: Failed to remove edge with " +
                                  "destination " + removedDestVertex + "on " +
                                  vertex + " since it doesn't exist.");
@@ -101,7 +103,7 @@ public class VertexResolver<I extends Wr
                 (MutableVertex<I, V, E, M>) vertex;
             for (Edge<I, E> edge : vertexChanges.getAddedEdgeList()) {
                 edge.setConf(getConf());
-                mutableVertex.addEdge(edge);
+                mutableVertex.addEdge(edge.getDestVertexId(), edge.getEdgeValue());
             }
         }
 

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexInputFormat.java Wed Sep 14 04:13:54 2011
@@ -18,13 +18,7 @@
 
 package org.apache.giraph.lib;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-
 import net.iharder.Base64;
-
 import org.apache.giraph.graph.BspUtils;
 import org.apache.giraph.graph.Edge;
 import org.apache.giraph.graph.MutableVertex;
@@ -41,6 +35,11 @@ import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
 /**
  * Simple way to represent the structure of the graph with a JSON object.
  * The actual vertex ids, values, edges are stored by the
@@ -138,7 +137,7 @@ public class JsonBase64VertexInputFormat
                 Edge<I, E> edge = new Edge<I, E>();
                 edge.setConf(getContext().getConfiguration());
                 edge.readFields(input);
-                vertex.addEdge(edge);
+                vertex.addEdge(edge.getDestVertexId(), edge.getEdgeValue());
             }
             return true;
         }

Modified: incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java
URL: http://svn.apache.org/viewvc/incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java?rev=1170431&r1=1170430&r2=1170431&view=diff
==============================================================================
--- incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java (original)
+++ incubator/giraph/trunk/src/main/java/org/apache/giraph/lib/JsonBase64VertexOutputFormat.java Wed Sep 14 04:13:54 2011
@@ -18,13 +18,7 @@
 
 package org.apache.giraph.lib;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
 import net.iharder.Base64;
-
 import org.apache.giraph.graph.BasicVertex;
 import org.apache.giraph.graph.Edge;
 import org.apache.giraph.graph.VertexWriter;
@@ -37,6 +31,11 @@ import org.json.JSONArray;
 import org.json.JSONException;
 import org.json.JSONObject;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
 /**
  * Simple way to represent the structure of the graph with a JSON object.
  * The actual vertex ids, values, edges are stored by the
@@ -99,7 +98,10 @@ public class JsonBase64VertexOutputForma
                     "writerVertex: Failed to insert vertex value", e);
             }
             JSONArray edgeArray = new JSONArray();
-            for (Edge<I, E> edge : vertex.getOutEdgeMap().values()) {
+            for (I targetVertexId : vertex) {
+                Edge<I, E> edge = new Edge<I, E>(
+                    targetVertexId, vertex.getEdgeValue(targetVertexId));
+                edge.setConf(getContext().getConfiguration());
                 outputStream.reset();
                 edge.write(output);
                 edgeArray.put(Base64.encodeBytes(outputStream.toByteArray()));