You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/01/14 07:59:25 UTC

[01/50] git commit: Finished docummenting join operators and revised some of the initial presentation.

Updated Branches:
  refs/heads/master 945fe7a37 -> 4a805aff5


Finished docummenting join operators and revised some of the initial presentation.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/64c45935
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/64c45935
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/64c45935

Branch: refs/heads/master
Commit: 64c4593586233409ff2c41607e7df33f3f13eb0a
Parents: 02771aa
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Sat Jan 11 13:48:24 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Sat Jan 11 13:48:35 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 119 +++++++++++++++++++++++-----------
 docs/img/graphx_figures.pptx     | Bin 1123365 -> 1123363 bytes
 2 files changed, 82 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/64c45935/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index b19c6b6..5c9f196 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -16,16 +16,14 @@ title: GraphX Programming Guide
 
 # Overview
 
-GraphX is the new (alpha) Spark API for graphs and graph-parallel
-computation. At a high-level, GraphX extends the Spark
-[RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the
-[Resilient Distributed property Graph (RDG)](#property_graph): a directed graph
-with properties attached to each vertex and edge.  To support graph computation,
-GraphX exposes a set of functions (e.g., [mapReduceTriplets](#mrTriplets)) as
-well as an optimized variant of the [Pregel](http://giraph.apache.org) API. In
-addition, GraphX includes a growing collection of graph
-[algorithms](#graph_algorithms) and [builders](#graph_builders) to simplify
-graph analytics tasks.
+GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high-level,
+GraphX extends the Spark [RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the
+[Resilient Distributed property Graph (RDG)](#property_graph): a directed multigraph with properties
+attached to each vertex and edge.  To support graph computation, GraphX exposes a set of functions
+(e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and
+[mapReduceTriplets](#mrTriplets)) as well as an optimized variant of the
+[Pregel](#pregel) API. In addition, GraphX includes a growing collection of graph
+[algorithms](#graph_algorithms) and [builders](#graph_builders) to simplify graph analytics tasks.
 
 ## Background on Graph-Parallel Computation
 
@@ -60,12 +58,10 @@ movement and duplication and a complicated programming model.
   <!-- Images are downsized intentionally to improve quality on retina displays -->
 </p>
 
-The goal of the GraphX project is to unify graph-parallel and data-parallel
-computation in one system with a single composable API. The GraphX API
-enables users to view data both as a graph and as
-collection (i.e., RDDs) without data movement or duplication. By
-incorporating recent advances in graph-parallel systems, GraphX is able to optimize
-the execution of graph operations.
+The goal of the GraphX project is to unify graph-parallel and data-parallel computation in one
+system with a single composable API. The GraphX API enables users to view data both as a graph and
+as collections (i.e., RDDs) without data movement or duplication. By incorporating recent advances
+in graph-parallel systems, GraphX is able to optimize the execution of graph operations.
 
 ## GraphX Replaces the Spark Bagel API
 
@@ -95,12 +91,16 @@ If you are not using the Spark shell you will also need a Spark context.
 <a name="property_graph"></a>
 
 The [property graph](api/graphx/index.html#org.apache.spark.graphx.Graph) is a directed multigraph
-graph with user defined objects attached to each vertex and edge.  As a multigraph it is possible
-for multiple edges to have the same source and destination vertex.  This can be useful when there
-are multiple relationships between the same vertices.  Like RDDs, property graphs are immutable,
-distributed, and fault-tolerant. Vertices are keyed by their vertex identifier (`VertexId`) which is
-a unique 64-bit long. Similarly, edges have corresponding source and destination vertex identifiers.
-Unlike other systems, GraphX does not impose any ordering or constraints on the vertex identifiers.
+graph with user defined objects attached to each vertex and edge.  A directed multigraph is a
+directed graph with potentially multiple parallel edges sharing the same source and destination
+vertex.  The ability to support parallel edges simplifies modeling scenarios where there can be
+multiple relationships (e.g., co-worker and friend) between the same vertices.  Note, however there
+can only be one instance of each vertex.
+
+Like RDDs, property graphs are immutable, distributed, and fault-tolerant. Vertices are keyed by
+their vertex identifier (`VertexId`) which is a unique 64-bit long. Similarly, edges have
+corresponding source and destination vertex identifiers. GraphX does not impose any ordering or
+constraints on the vertex identifiers.
 
 The property graph is parameterized over the vertex `VD` and edge `ED` types.  These are the types
 of the objects associated with each vertex and edge respectively.  In some cases it can be desirable
@@ -119,9 +119,12 @@ class Graph[VD: ClassTag, ED: ClassTag] {
   // ...
 }
 {% endhighlight %}
+
 > Note that the vertices and edges of the graph are actually of type `VertexRDD[VD]` and
-> `EdgeRDD[ED]` respectively. These types extend and are optimized versions of `RDD[(VertexId, VD)]`
-> and `RDD[Edge[ED]]`.
+> `EdgeRDD[ED]` respectively. These classes extend and are optimized versions of `RDD[(VertexId,
+> VD)]` and `RDD[Edge[ED]]` with additional functionality built around the internal index and column
+> oriented representations.  We discuss the `VertexRDD` and `EdgeRDD` API in greater detail in the
+> section on [vertex and edge RDDs](#vertex_and_edge_rdds)
 
 For example, we might construct a property graph consisting of various collaborators on the GraphX
 project. The vertex property contains the username and occupation and the edge property contains
@@ -259,7 +262,7 @@ defined `map` function.
 > Note that in all cases the graph structure is unaffected.  This is a key feature of these
 > operators which allows the resulting graph to reuse the structural indicies and the unaffected
 > properties of the original graph.
-> While `graph.mapVertices(mapUDF)` is logically equivalent to the following, the following
+> While the following is logically equivalent to `graph.mapVertices(mapUDF)`, it
 > does not preserve the structural indicies and would not benefit from the substantial system
 > optimizations in GraphX.
 > {% highlight scala %}
@@ -340,32 +343,74 @@ thereby reducing the graph size in memory as well as the cost of computation.
 ## Join Operators
 <a name="join_operators"></a>
 
-The ability to move between graph and collection views of data is a key part of GraphX.  In many
-cases it is necessary to bring data from external collections into the graph.  For example, we might
-have extra user properties that we want to merge with an existing graph or we might want to pull
-vertex properties from one graph into another.  These tasks can be accomplished using the *join*
-operators.  Below we list the key join operators:
+The ability to move between graph and collection views is a key part of GraphX.  In many cases it is
+necessary to join data from external collections (RDDs) with graphs.  For example, we might have
+extra user properties that we want to merge with an existing graph or we might want to pull vertex
+properties from one graph into another.  These tasks can be accomplished using the *join* operators.
+Below we list the key join operators:
 
 {% highlight scala %}
-def joinVertices[U](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, U) => VD)
+def joinVertices[U](table: RDD[(VertexID, U)])(map: (VertexID, VD, U) => VD)
   : Graph[VD, ED]
-def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, Option[U]) => VD2)
+def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(map: (VertexID, VD, Option[U]) => VD2)
   : Graph[VD2, ED]
 {% endhighlight %}
 
-## Map Reduce Triplets (mapReduceTriplets)
-<a name="mrTriplets"></a>
+The `joinVertices` operators, defined in
+[`GraphOps.scala`](api/graphx/index.html#org.apache.spark.graphx.GraphOps), joins the vertices with
+the input RDD and returns a new graph with the vertex properties obtained by applying the user
+defined `map` function to the result of the joined vertices.  Vertices without a matching value in
+the RDD retain their original value.
 
+> Note that if the RDD contains more than one value for a given vertex only one will be used.   It
+> is therefore recommended that the input RDD be first made unique using the following which will
+> also *pre-index* the resulting values to substantially accelerate the subsequent join.
+> {% highlight scala %}
+val nonUniqueCosts: RDD[(VertexId, Double)]
+val uniqueCosts: VertexRDD[Double] =
+  graph.vertices.aggregateUsingIndex(nonUnique, (a,b) => a + b)
+val joinedGraph = graph.joinVertices(uniqueCosts)(
+  (id, oldCost, extraCost) => oldCost + extraCost)
+{% endhighlight %}
 
+The more general `outerJoinVertices` behaves similarly to `joinVertices` except that the user
+defined `map` function is applied to all vertices and can change the vertex property type.  Because
+not all vertices may have a matching value in the input RDD the `map` function takes an `Option`
+type.  For example, we can setup a graph for PageRank by initializing vertex properties with their
+`outDegree`.
 
+{% highlight scala %}
+val outDegrees: VertexRDD[Int] = graph.outDegrees
+val degreeGraph = graph.outerJoinVertices(outDegrees) { (id, oldAttr, outDegOpt) =>
+  outDegOpt match {
+    case Some(outDeg) => outDeg
+    case None => 0 // No outDegree means zero outDegree
+  }
+}
+{% endhighlight %}
+
+> You may have noticed the multiple parameter lists (e.g., `f(a)(b)`) curried function pattern used
+> in the above examples.  While we could have equally written `f(a)(b)` as `f(a,b)` this would mean
+> that type inference on `b` would not depend on `a`.  As a consequence, the user would need to
+> provide type annotation for the user defined function:
+> {% highlight scala %}
+val joinedGraph = graph.joinVertices(uniqueCosts,
+  (id: VertexId, oldCost: Double, extraCost: Double) => oldCost + extraCost)
+{% endhighlight %}
+
+
+## Map Reduce Triplets (mapReduceTriplets)
+<a name="mrTriplets"></a>
+
+# Pregel API
+<a name="pregel"></a>
 
 # Graph Builders
 <a name="graph_builders"></a>
 
+# Vertex and Edge RDDs
+<a name="vertex_and_edge_rdds"></a>
 
-{% highlight scala %}
-val userGraph: Graph[(String, String), String]
-{% endhighlight %}
 
 
 # Optimized Representation

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/64c45935/docs/img/graphx_figures.pptx
----------------------------------------------------------------------
diff --git a/docs/img/graphx_figures.pptx b/docs/img/graphx_figures.pptx
index ea4f82c..e567bf0 100644
Binary files a/docs/img/graphx_figures.pptx and b/docs/img/graphx_figures.pptx differ


[46/50] git commit: Finish 6f6f8c928ce493357d4d32e46971c5e401682ea8

Posted by pw...@apache.org.
Finish 6f6f8c928ce493357d4d32e46971c5e401682ea8


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/2cd9358c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/2cd9358c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/2cd9358c

Branch: refs/heads/master
Commit: 2cd9358ccf28186e88016b6542d7c0c90536651f
Parents: 76ebdae
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 22:29:23 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 22:29:23 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2cd9358c/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 29d397c..226299a 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -125,8 +125,10 @@ properties for each vertex and edge.  As a consequence, the graph class contains
 the vertices and edges of the graph:
 
 {% highlight scala %}
-val vertices: VertexRDD[VD]
-val edges: EdgeRDD[ED]
+class Graph[VD, ED] {
+  val vertices: VertexRDD[VD]
+  val edges: EdgeRDD[ED]
+}
 {% endhighlight %}
 
 The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexId,


[11/50] git commit: Tested and corrected all examples up to mask in the graphx-programming-guide.

Posted by pw...@apache.org.
Tested and corrected all examples up to mask in the graphx-programming-guide.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/66c9d009
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/66c9d009
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/66c9d009

Branch: refs/heads/master
Commit: 66c9d0092ae28e07c4fae8b026cca6cf74f1c37a
Parents: 1efe78a
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Sun Jan 12 22:11:04 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Sun Jan 12 22:11:13 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 37 +++++++++++++++++++----------------
 1 file changed, 20 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/66c9d009/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index f2f5a88..2697b2d 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -80,6 +80,8 @@ To get started you first need to import Spark and GraphX into your project, as f
 {% highlight scala %}
 import org.apache.spark._
 import org.apache.spark.graphx._
+// To make some of the examples work we will also need RDD
+import org.apache.spark.rdd.RDD
 {% endhighlight %}
 
 If you are not using the Spark shell you will also need a Spark context.
@@ -105,13 +107,11 @@ be accomplished through inheritance.  For example to model users and products as
 we might do the following:
 
 {% highlight scala %}
-case class VertexProperty
-case class UserProperty extends VertexProperty
-  (val name: String)
-case class ProductProperty extends VertexProperty
-  (val name: String, val price: Double)
+class VertexProperty()
+case class UserProperty(val name: String) extends VertexProperty
+case class ProductProperty(val name: String, val price: Double) extends VertexProperty
 // The graph might then have the type:
-val graph: Graph[VertexProperty, String]
+var graph: Graph[VertexProperty, String] = null
 {% endhighlight %}
 
 Like RDDs, property graphs are immutable, distributed, and fault-tolerant.  Changes to the values or
@@ -165,13 +165,13 @@ code constructs a graph from a collection of RDDs:
 // Assume the SparkContext has already been constructed
 val sc: SparkContext
 // Create an RDD for the vertices
-val users: RDD[(VertexId, (String, String))] =
-  sc.parallelize(Array((3, ("rxin", "student")), (7, ("jgonzal", "postdoc")),
-                       (5, ("franklin", "prof")), (2, ("istoica", "prof"))))
+val users: RDD[(VertexID, (String, String))] =
+  sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
+                       (5L, ("franklin", "prof")), (2L, ("istoica", "prof"))))
 // Create an RDD for edges
 val relationships: RDD[Edge[String]] =
-  sc.parallelize(Array(Edge(3, 7, "collab"), Edge(5, 3, "advisor"),
-                       Edge(2, 5, "colleague"), Edge(5, 7, "pi"))
+  sc.parallelize(Array(Edge(3L, 7L, "collab"),    Edge(5L, 3L, "advisor"),
+                       Edge(2L, 5L, "colleague"), Edge(5L, 7L, "pi")))
 // Define a default user in case there are relationship with missing user
 val defaultUser = ("John Doe", "Missing")
 // Build the initial Graph
@@ -200,7 +200,7 @@ graph.edges.filter(e => e.srcId > e.dstId).count
 > tuple.  On the other hand, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects.
 > We could have also used the case class type constructor as in the following:
 > {% highlight scala %}
-graph.edges.filter { case Edge(src, dst, prop) => src < dst }.count
+graph.edges.filter { case Edge(src, dst, prop) => src > dst }.count
 {% endhighlight %}
 
 In addition to the vertex and edge views of the property graph, GraphX also exposes a triplet view.
@@ -234,7 +234,9 @@ triplet view of a graph to render a collection of strings describing relationshi
 val graph: Graph[(String, String), String] // Constructed from above
 // Use the triplets view to create an RDD of facts.
 val facts: RDD[String] =
-  graph.triplets.map(et => et.srcAttr._1 + " is the " + et.attr + " of " et.dstAttr)
+  graph.triplets.map(triplet =>
+    triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1)
+facts.collect.foreach(println(_))
 {% endhighlight %}
 
 # Graph Operators
@@ -294,11 +296,12 @@ unnecessary properties.  For example, given a graph with the out-degrees as the
 
 {% highlight scala %}
 // Given a graph where the vertex property is the out-degree
-val inputGraph: Graph[Int, String]
+val inputGraph: Graph[Int, String] =
+  graph.outerJoinVertices(graph.outDegrees)((vid, _, degOpt) => degOpt.getOrElse(0))
 // Construct a graph where each edge contains the weight
 // and each vertex is the initial PageRank
 val outputGraph: Graph[Double, Double] =
-  inputGraph.mapTriplets(et => 1.0 / et.srcAttr).mapVertices(v => 1.0)
+  inputGraph.mapTriplets(triplet => 1.0 / triplet.srcAttr).mapVertices((id, _) => 1.0)
 {% endhighlight %}
 
 ## Structural Operators
@@ -338,7 +341,7 @@ val defaultUser = ("John Doe", "Missing")
 // Build the initial Graph
 val graph = Graph(users, relationships, defaultUser)
 // Remove missing vertices as well as the edges to connected to them
-val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing")
+val validGraph = graph.subgraph(vpred = (id, attr) => attr._2 != "Missing")
 {% endhighlight %}
 
 > Note in the above example only the vertex predicate is provided.  The `subgraph` operator defaults
@@ -356,7 +359,7 @@ the answer to the valid subgraph.
 // Run Connected Components
 val ccGraph = graph.connectedComponents() // No longer contains missing field
 // Remove missing vertices as well as the edges to connected to them
-val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing")
+val validGraph = graph.subgraph(vpred = (id, attr) => attr._2 != "Missing")
 // Restrict the answer to the valid subgraph
 val validCCGraph = ccGraph.mask(validGraph)
 {% endhighlight %}


[49/50] git commit: Adding minimal additional functionality to EdgeRDD

Posted by pw...@apache.org.
Adding minimal additional functionality to EdgeRDD


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/80e73ed0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/80e73ed0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/80e73ed0

Branch: refs/heads/master
Commit: 80e73ed0004cceb47a450c79aa4faa598502fa45
Parents: 4bafc4f
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Mon Jan 13 22:56:57 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Mon Jan 13 22:56:57 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/graphx/EdgeRDD.scala    | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/80e73ed0/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
index c0a23d1..832b781 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
@@ -58,6 +58,23 @@ class EdgeRDD[@specialized ED: ClassTag](
   }
 
   /**
+   * Map the values in an edge partitioning preserving the structure but changing the values.
+   *
+   * @tparam ED2 the new edge value type
+   * @param f the function from an edge to a new edge value
+   * @return a new EdgeRDD containing the new edge values
+   */
+  def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2] =
+    mapEdgePartitions((pid, part) => part.map(f))
+
+  /**
+   * Reverse all the edges in this RDD.
+   *
+   * @return a new EdgeRDD containing all the edges reversed
+   */
+  def reverse: EdgeRDD[ED] = mapEdgePartitions((pid, part) => part.reverse)
+
+  /**
    * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same
    * [[PartitionStrategy]].
    *


[37/50] git commit: Make Graph{, Impl, Ops} serializable to work around capture

Posted by pw...@apache.org.
Make Graph{,Impl,Ops} serializable to work around capture


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/84d6af80
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/84d6af80
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/84d6af80

Branch: refs/heads/master
Commit: 84d6af80211bbe15777a8d48eb37052b0b6631c4
Parents: d4d9ece
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 20:59:45 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:02:37 2014 -0800

----------------------------------------------------------------------
 graphx/src/main/scala/org/apache/spark/graphx/Graph.scala          | 2 +-
 graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala       | 2 +-
 graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/84d6af80/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 7e99276..9dd05ad 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -20,7 +20,7 @@ import org.apache.spark.storage.StorageLevel
  * @tparam VD the vertex attribute type
  * @tparam ED the edge attribute type
  */
-abstract class Graph[VD: ClassTag, ED: ClassTag] {
+abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializable {
 
   /**
    * An RDD containing the vertices and their associated attributes.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/84d6af80/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 447ef55..233adb9 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -14,7 +14,7 @@ import org.apache.spark.rdd.RDD
  * @tparam VD the vertex attribute type
  * @tparam ED the edge attribute type
  */
-class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
+class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Serializable {
 
   /** The number of edges in the graph. */
   lazy val numEdges: Long = graph.edges.count()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/84d6af80/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 12d46a8..56d1d9e 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -32,7 +32,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
     @transient val replicatedVertexView: ReplicatedVertexView[VD])
   extends Graph[VD, ED] with Serializable {
 
-  /** Default construct is provided to support serialization */
+  /** Default constructor is provided to support serialization */
   protected def this() = this(null, null, null, null)
 
   /** Return a RDD that brings edges together with their source and destination vertices. */


[03/50] git commit: Link methods in programming guide; document VertexID

Posted by pw...@apache.org.
Link methods in programming guide; document VertexID


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/f096f4ea
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/f096f4ea
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/f096f4ea

Branch: refs/heads/master
Commit: f096f4eaf1f8e936eafc2006ecd01faa2f208cf2
Parents: cf57b1b
Author: Ankur Dave <an...@gmail.com>
Authored: Sun Jan 12 10:55:29 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Sun Jan 12 10:55:29 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 155 ++++++++++---------
 .../scala/org/apache/spark/graphx/package.scala |   4 +
 2 files changed, 90 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f096f4ea/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 9a7c4ac..7f93754 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -68,15 +68,14 @@ in graph-parallel systems, GraphX is able to optimize the execution of graph ope
 Prior to the release of GraphX, graph computation in Spark was expressed using Bagel, an
 implementation of Pregel.  GraphX improves upon Bagel by exposing a richer property graph API, a
 more streamlined version of the Pregel abstraction, and system optimizations to improve performance
-and reduce memory overhead.  While we plan to eventually deprecate the Bagel, we will continue to
-support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and [Bagel programming
-guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX
-API and comment on issues that may complicate the transition from Bagel.
+and reduce memory overhead.  While we plan to eventually deprecate Bagel, we will continue to
+support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and
+[Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to
+explore the new GraphX API and comment on issues that may complicate the transition from Bagel.
 
 # Getting Started
 
-To get started you first need to import Spark and GraphX into your project.  This can be done by
-importing the following:
+To get started you first need to import Spark and GraphX into your project, as follows:
 
 {% highlight scala %}
 import org.apache.spark._
@@ -89,11 +88,11 @@ If you are not using the Spark shell you will also need a Spark context.
 <a name="property_graph"></a>
 
 The [property graph](api/graphx/index.html#org.apache.spark.graphx.Graph) is a directed multigraph
-graph with user defined objects attached to each vertex and edge.  A directed multigraph is a
-directed graph with potentially multiple parallel edges sharing the same source and destination
-vertex.  The ability to support parallel edges simplifies modeling scenarios where there can be
-multiple relationships (e.g., co-worker and friend) between the same vertices.  Each vertex is keyed
-by a *unique* 64-bit long identifier (`VertexId`).  Similarly, edges have corresponding source and
+with user defined objects attached to each vertex and edge.  A directed multigraph is a directed
+graph with potentially multiple parallel edges sharing the same source and destination vertex.  The
+ability to support parallel edges simplifies modeling scenarios where there can be multiple
+relationships (e.g., co-worker and friend) between the same vertices.  Each vertex is keyed by a
+*unique* 64-bit long identifier (`VertexId`).  Similarly, edges have corresponding source and
 destination vertex identifiers. GraphX does not impose any ordering or constraints on the vertex
 identifiers.  The property graph is parameterized over the vertex `VD` and edge `ED` types.  These
 are the types of the objects associated with each vertex and edge respectively.
@@ -102,8 +101,8 @@ are the types of the objects associated with each vertex and edge respectively.
 > int, double, etc...) reducing the in memory footprint.
 
 In some cases we may wish to have vertices with different property types in the same graph. This can
-be accomplished through inheritance.  For example to model users and products as a bipartie graph we
-might do the following:
+be accomplished through inheritance.  For example to model users and products as a bipartite graph
+we might do the following:
 
 {% highlight scala %}
 case class VertexProperty
@@ -159,8 +158,8 @@ val userGraph: Graph[(String, String), String]
 There are numerous ways to construct a property graph from raw files, RDDs, and even synthetic
 generators and these are discussed in more detail in the section on
 [graph builders](#graph_builders).  Probably the most general method is to use the
-[graph singleton](api/graphx/index.html#org.apache.spark.graphx.Graph$).
-For example the following code constructs a graph from a collection of RDDs:
+[Graph object](api/graphx/index.html#org.apache.spark.graphx.Graph$).  For example the following
+code constructs a graph from a collection of RDDs:
 
 {% highlight scala %}
 // Assume the SparkContext has already been constructed
@@ -179,10 +178,11 @@ val defaultUser = ("John Doe", "Missing")
 val graph = Graph(users, relationships, defaultUser)
 {% endhighlight %}
 
-In the above example we make use of the [`Edge`](api/graphx/index.html#org.apache.spark.graphx.Edge)
-case class. Edges have a `srcId` and a `dstId` corresponding to the source and destination vertex
-identifiers. In addition, the `Edge` class contains the `attr` member which contains the edge
-property.
+In the above example we make use of the [`Edge`][Edge] case class. Edges have a `srcId` and a
+`dstId` corresponding to the source and destination vertex identifiers. In addition, the `Edge`
+class contains the `attr` member which contains the edge property.
+
+[Edge]: api/graphx/index.html#org.apache.spark.graphx.Edge
 
 We can deconstruct a graph into the respective vertex and edge views by using the `graph.vertices`
 and `graph.edges` members respectively.
@@ -196,18 +196,19 @@ graph.edges.filter(e => e.srcId > e.dstId).count
 {% endhighlight %}
 
 > Note that `graph.vertices` returns an `VertexRDD[(String, String)]` which extends
-> `RDD[(VertexId, (String, String))]` and so we use the scala `case` expression to deconstruct
-> the tuple.  Alternatively, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects.
+> `RDD[(VertexId, (String, String))]` and so we use the scala `case` expression to deconstruct the
+> tuple.  On the other hand, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects.
 > We could have also used the case class type constructor as in the following:
 > {% highlight scala %}
 graph.edges.filter { case Edge(src, dst, prop) => src < dst }.count
 {% endhighlight %}
 
 In addition to the vertex and edge views of the property graph, GraphX also exposes a triplet view.
-The triplet view logically joins the vertex and edge properties yielding an `RDD[EdgeTriplet[VD,
-ED]]` containing instances of the
-[`EdgeTriplet`](api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet) class. This *join* can be
-expressed in the following SQL expression:
+The triplet view logically joins the vertex and edge properties yielding an
+`RDD[EdgeTriplet[VD, ED]]` containing instances of the [`EdgeTriplet`][EdgeTriplet] class. This
+*join* can be expressed in the following SQL expression:
+
+[EdgeTriplet]: api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet
 
 {% highlight sql %}
 SELECT src.id, dst.id, src.attr, e.attr, dst.attr
@@ -225,8 +226,7 @@ or graphically as:
   <!-- Images are downsized intentionally to improve quality on retina displays -->
 </p>
 
-The [`EdgeTriplet`](api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet) class extends the
-[`Edge`](api/graphx/index.html#org.apache.spark.graphx.Edge) class by adding the `srcAttr` and
+The [`EdgeTriplet`][EdgeTriplet] class extends the [`Edge`][Edge] class by adding the `srcAttr` and
 `dstAttr` members which contain the source and destination properties respectively. We can use the
 triplet view of a graph to render a collection of strings describing relationships between users.
 
@@ -240,14 +240,15 @@ val facts: RDD[String] =
 # Graph Operators
 
 Just as RDDs have basic operations like `map`, `filter`, and `reduceByKey`, property graphs also
-have a collection of basic operators that take user defined function and produce new graphs with
+have a collection of basic operators that take user defined functions and produce new graphs with
 transformed properties and structure.  The core operators that have optimized implementations are
-defined in [`Graph.scala`](api/graphx/index.html#org.apache.spark.graphx.Graph) and convenient
-operators that are expressed as a compositions of the core operators are defined in
-['GraphOps.scala'](api/graphx/index.html#org.apache.spark.graphx.GraphOps).  However, thanks to
-Scala implicits the operators in `GraphOps.scala` are automatically available as members of
-`Graph.scala`.  For example, we can compute the in-degree of each vertex (defined in
-'GraphOps.scala') by the following:
+defined in [`Graph`][Graph] and convenient operators that are expressed as a compositions of the
+core operators are defined in [`GraphOps`][GraphOps].  However, thanks to Scala implicits the
+operators in `GraphOps` are automatically available as members of `Graph`.  For example, we can
+compute the in-degree of each vertex (defined in `GraphOps`) by the following:
+
+[Graph]: api/graphx/index.html#org.apache.spark.graphx.Graph
+[GraphOps]: api/graphx/index.html#org.apache.spark.graphx.GraphOps
 
 {% highlight scala %}
 val graph: Graph[(String, String), String]
@@ -272,20 +273,24 @@ def mapTriplets[ED2](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2]
 Each of these operators yields a new graph with the vertex or edge properties modified by the user
 defined `map` function.
 
-> Note that in all cases the graph structure is unaffected.  This is a key feature of these
-> operators which allows the resulting graph to reuse the structural indicies and the unaffected
-> properties of the original graph.
-> While the following is logically equivalent to `graph.mapVertices(mapUDF)`, it
-> does not preserve the structural indicies and would not benefit from the substantial system
-> optimizations in GraphX.
+> Note that in all cases the graph structure is unaffected. This is a key feature of these operators
+> which allows the resulting graph to reuse the structural indices of the original graph. The
+> following snippets are logically equivalent, but the first one does not preserve the structural
+> indices and would not benefit from the GraphX system optimizations:
 > {% highlight scala %}
 val newVertices = graph.vertices.map { case (id, attr) => (id, mapUdf(id, attr)) }
 val newGraph = Graph(newVertices, graph.edges)
 {% endhighlight %}
+> Instead, use [`mapVertices`][Graph.mapVertices] to preserve the indices:
+> {% highlight scala %}
+val newGraph = graph.mapVertices((id, attr) => mapUdf(id, attr))
+{% endhighlight %}
+
+[Graph.mapVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexID,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED]
 
 These operators are often used to initialize the graph for a particular computation or project away
 unnecessary properties.  For example, given a graph with the out-degrees as the vertex properties
-(we describe how to construct such a graph later) we initialize for PageRank:
+(we describe how to construct such a graph later), we initialize it for PageRank:
 
 {% highlight scala %}
 // Given a graph where the vertex property is the out-degree
@@ -293,7 +298,7 @@ val inputGraph: Graph[Int, String]
 // Construct a graph where each edge contains the weight
 // and each vertex is the initial PageRank
 val outputGraph: Graph[Double, Double] =
-  inputGraph.mapTriplets(et => 1.0/et.srcAttr).mapVertices(v => 1.0)
+  inputGraph.mapTriplets(et => 1.0 / et.srcAttr).mapVertices(v => 1.0)
 {% endhighlight %}
 
 ## Structural Operators
@@ -310,16 +315,20 @@ def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED]
 def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED]
 {% endhighlight %}
 
-The `reverse` operator returns a new graph with all the edge directions reversed.  This can be
-useful when, for example, trying to compute the inverse PageRank.  Because the reverse operation
-does not modify vertex or edge properties or change the number of edges, it can be implemented
-efficiently without data-movement or duplication.
+The [`reverse`][Graph.reverse] operator returns a new graph with all the edge directions reversed.
+This can be useful when, for example, trying to compute the inverse PageRank.  Because the reverse
+operation does not modify vertex or edge properties or change the number of edges, it can be
+implemented efficiently without data-movement or duplication.
+
+[Graph.reverse]: api/graphx/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED]
 
-The `subgraph` operator takes vertex and edge predicates and returns the graph containing only the
-vertices that satisfy the vertex predicate (evaluate to true) and edges that satisfy the edge
-predicate *and connect vertices that satisfy the vertex predicate*.  The `subgraph` operator can be
-used in number of situations to restrict the graph to the vertices and edges of interest or
-eliminate broken links. For example in the following code we remove broken links:
+The [`subgraph`][Graph.subgraph] operator takes vertex and edge predicates and returns the graph
+containing only the vertices that satisfy the vertex predicate (evaluate to true) and edges that
+satisfy the edge predicate *and connect vertices that satisfy the vertex predicate*.  The `subgraph`
+operator can be used in number of situations to restrict the graph to the vertices and edges of
+interest or eliminate broken links. For example in the following code we remove broken links:
+
+[Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexID,VD)⇒Boolean):Graph[VD,ED]
 
 {% highlight scala %}
 val users: RDD[(VertexId, (String, String))]
@@ -335,11 +344,13 @@ val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing")
 > Note in the above example only the vertex predicate is provided.  The `subgraph` operator defaults
 > to `true` if the vertex or edge predicates are not provided.
 
-The `mask` operator also constructs a subgraph by returning a graph that contains the vertices and
-edges that are also found in the input graph.  This can be used in conjunction with the `subgraph`
-operator to restrict a graph based on the properties in another related graph.  For example, we
-might run connected components using the graph with missing vertices and then restrict the answer to
-the valid subgraph.
+The [`mask`][Graph.mask] operator also constructs a subgraph by returning a graph that contains the
+vertices and edges that are also found in the input graph.  This can be used in conjunction with the
+`subgraph` operator to restrict a graph based on the properties in another related graph.  For
+example, we might run connected components using the graph with missing vertices and then restrict
+the answer to the valid subgraph.
+
+[Graph.mask]: api/graphx/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED]
 
 {% highlight scala %}
 // Run Connected Components
@@ -350,9 +361,11 @@ val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing")
 val validCCGraph = ccGraph.mask(validGraph)
 {% endhighlight %}
 
-The `groupEdges` operator merges parallel edges (i.e., duplicate edges between pairs of vertices) in
-the multigraph.  In many numerical applications, parallel edges can be *added* (their weights
-combined) into a single edge thereby reducing the size of the graph.
+The [`groupEdges`][Graph.groupEdges] operator merges parallel edges (i.e., duplicate edges between
+pairs of vertices) in the multigraph.  In many numerical applications, parallel edges can be *added*
+(their weights combined) into a single edge thereby reducing the size of the graph.
+
+[Graph.groupEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED]
 
 ## Join Operators
 <a name="join_operators"></a>
@@ -369,11 +382,12 @@ def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(map: (VertexID, VD, Opt
   : Graph[VD2, ED]
 {% endhighlight %}
 
-The `joinVertices` operator, defined in
-[`GraphOps.scala`](api/graphx/index.html#org.apache.spark.graphx.GraphOps), joins the vertices with
-the input RDD and returns a new graph with the vertex properties obtained by applying the user
-defined `map` function to the result of the joined vertices.  Vertices without a matching value in
-the RDD retain their original value.
+The [`joinVertices`][GraphOps.joinVertices] operator joins the vertices with the input RDD and
+returns a new graph with the vertex properties obtained by applying the user defined `map` function
+to the result of the joined vertices.  Vertices without a matching value in the RDD retain their
+original value.
+
+[GraphOps.joinVertices]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexID,U)])((VertexID,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED]
 
 > Note that if the RDD contains more than one value for a given vertex only one will be used.   It
 > is therefore recommended that the input RDD be first made unique using the following which will
@@ -386,11 +400,14 @@ val joinedGraph = graph.joinVertices(uniqueCosts)(
   (id, oldCost, extraCost) => oldCost + extraCost)
 {% endhighlight %}
 
-The more general `outerJoinVertices` behaves similarly to `joinVertices` except that the user
-defined `map` function is applied to all vertices and can change the vertex property type.  Because
-not all vertices may have a matching value in the input RDD the `map` function takes an `Option`
-type.  For example, we can setup a graph for PageRank by initializing vertex properties with their
-`outDegree`.
+The more general [`outerJoinVertices`][Graph.outerJoinVertices] behaves similarly to `joinVertices`
+except that the user defined `map` function is applied to all vertices and can change the vertex
+property type.  Because not all vertices may have a matching value in the input RDD the `map`
+function takes an `Option` type.  For example, we can setup a graph for PageRank by initializing
+vertex properties with their `outDegree`.
+
+[Graph.outerJoinVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexID,U)])((VertexID,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED]
+
 
 {% highlight scala %}
 val outDegrees: VertexRDD[Int] = graph.outDegrees

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f096f4ea/graphx/src/main/scala/org/apache/spark/graphx/package.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala
index 2501314..e70d2fd 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala
@@ -3,6 +3,10 @@ package org.apache.spark
 import org.apache.spark.util.collection.OpenHashSet
 
 package object graphx {
+  /**
+   * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need
+   * to follow any ordering or any constraints other than uniqueness.
+   */
   type VertexID = Long
 
   // TODO: Consider using Char.


[25/50] git commit: Merge branch 'graphx' of github.com:ankurdave/incubator-spark into graphx

Posted by pw...@apache.org.
Merge branch 'graphx' of github.com:ankurdave/incubator-spark into graphx

Conflicts:
	graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/a4e12af7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/a4e12af7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/a4e12af7

Branch: refs/heads/master
Commit: a4e12af7aa7324653b7c65fc3c1c0454333520bf
Parents: 02a8f54 cfe4a29
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 17:42:59 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 17:42:59 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 56 +++++-------
 .../org/apache/spark/graphx/EdgeDirection.scala |  8 +-
 .../scala/org/apache/spark/graphx/Graph.scala   |  9 +-
 .../org/apache/spark/graphx/GraphOps.scala      | 91 +++++---------------
 .../scala/org/apache/spark/graphx/Pregel.scala  |  9 +-
 .../apache/spark/graphx/impl/GraphImpl.scala    |  7 ++
 .../spark/graphx/lib/ConnectedComponents.scala  | 41 +++------
 .../org/apache/spark/graphx/lib/PageRank.scala  |  5 +-
 .../lib/StronglyConnectedComponents.scala       |  3 +-
 .../apache/spark/graphx/lib/TriangleCount.scala |  2 +-
 .../org/apache/spark/graphx/GraphOpsSuite.scala | 28 +-----
 .../org/apache/spark/graphx/PregelSuite.scala   |  2 +-
 .../graphx/lib/ConnectedComponentsSuite.scala   |  2 +-
 13 files changed, 92 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a4e12af7/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a4e12af7/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
----------------------------------------------------------------------
diff --cc graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
index ce4eb53,75b44dd..fc18f7e
--- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
@@@ -89,16 -92,14 +92,16 @@@ object Pregel 
     *
     */
    def apply[VD: ClassTag, ED: ClassTag, A: ClassTag]
 -    (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue,
 -      activeDirection: EdgeDirection = EdgeDirection.Either)(
 -      vprog: (VertexID, VD, A) => VD,
 -      sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
 +     (graph: Graph[VD, ED],
 +      initialMsg: A,
 +      maxIterations: Int = Int.MaxValue,
-       activeDirection: EdgeDirection = EdgeDirection.Out)
++      activeDirection: EdgeDirection = EdgeDirection.Either)
 +     (vprog: (VertexID, VD, A) => VD,
-       sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
++      sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
        mergeMsg: (A, A) => A)
 -    : Graph[VD, ED] = {
 -
 -    var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache()
 +    : Graph[VD, ED] =
 +  {
 +    var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)).cache()
      // compute the messages
      var messages = g.mapReduceTriplets(sendMsg, mergeMsg)
      var activeMessages = messages.count()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a4e12af7/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a4e12af7/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a4e12af7/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a4e12af7/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a4e12af7/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
----------------------------------------------------------------------


[09/50] git commit: Move algorithms to GraphOps

Posted by pw...@apache.org.
Move algorithms to GraphOps


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/d691e9f4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/d691e9f4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/d691e9f4

Branch: refs/heads/master
Commit: d691e9f47ed9b43b422712047183142d01c5e8c2
Parents: 20c509b
Author: Ankur Dave <an...@gmail.com>
Authored: Sun Jan 12 21:47:16 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Sun Jan 12 21:47:16 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 12 +---
 .../scala/org/apache/spark/graphx/Graph.scala   |  4 +-
 .../org/apache/spark/graphx/GraphOps.scala      | 51 ++++++++++++++-
 .../apache/spark/graphx/lib/Algorithms.scala    | 66 --------------------
 .../org/apache/spark/graphx/lib/package.scala   |  8 ---
 5 files changed, 54 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d691e9f4/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 0e228d8..572afc1 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -667,9 +667,7 @@ things to worry about.)
 # Graph Algorithms
 <a name="graph_algorithms"></a>
 
-GraphX includes a set of graph algorithms in to simplify analytics. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via an implicit conversion to [`Algorithms`][Algorithms]. This section describes the algorithms and how they are used.
-
-[Algorithms]: api/graphx/index.html#org.apache.spark.graphx.lib.Algorithms
+GraphX includes a set of graph algorithms in to simplify analytics. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via [`GraphOps`][GraphOps]. This section describes the algorithms and how they are used.
 
 ## PageRank
 <a name="pagerank"></a>
@@ -681,8 +679,6 @@ GraphX comes with static and dynamic implementations of PageRank as methods on t
 [PageRank]: api/graphx/index.html#org.apache.spark.graphx.lib.PageRank$
 
 {% highlight scala %}
-// Load the implicit conversion to Algorithms
-import org.apache.spark.graphx.lib._
 // Load the datasets into a graph
 val users = sc.textFile("graphx/data/users.txt").map { line =>
   val fields = line.split("\\s+")
@@ -710,8 +706,7 @@ The connected components algorithm labels each connected component of the graph
 [ConnectedComponents]: api/graphx/index.html#org.apache.spark.graphx.lib.ConnectedComponents$
 
 {% highlight scala %}
-// Load the implicit conversion and graph as in the PageRank example
-import org.apache.spark.graphx.lib._
+// Load the graph as in the PageRank example
 val users = ...
 val followers = ...
 val graph = Graph(users, followers)
@@ -733,8 +728,7 @@ A vertex is part of a triangle when it has two adjacent vertices with an edge be
 [Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph@partitionBy(PartitionStrategy):Graph[VD,ED]
 
 {% highlight scala %}
-// Load the implicit conversion and graph as in the PageRank example
-import org.apache.spark.graphx.lib._
+// Load the graph as in the PageRank example
 val users = ...
 // Load the edges in canonical order and partition the graph for triangle count
 val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(RandomVertexCut)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d691e9f4/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 56513ca..7d4f0de 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -15,9 +15,7 @@ import org.apache.spark.storage.StorageLevel
  * RDDs, the graph is a functional data-structure in which mutating
  * operations return new graphs.
  *
- * @note [[GraphOps]] contains additional convenience operations.
- * [[lib.Algorithms]] contains graph algorithms; to access these,
- * import `org.apache.spark.graphx.lib._`.
+ * @note [[GraphOps]] contains additional convenience operations and graph algorithms.
  *
  * @tparam VD the vertex attribute type
  * @tparam ED the edge attribute type

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d691e9f4/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 4fdff29..2b3b95e 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -2,9 +2,10 @@ package org.apache.spark.graphx
 
 import scala.reflect.ClassTag
 
-import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext._
 import org.apache.spark.SparkException
+import org.apache.spark.graphx.lib._
+import org.apache.spark.rdd.RDD
 
 /**
  * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the
@@ -298,4 +299,52 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
     Pregel(graph, initialMsg, maxIterations, activeDirection)(vprog, sendMsg, mergeMsg)
   }
 
+  /**
+   * Run a dynamic version of PageRank returning a graph with vertex attributes containing the
+   * PageRank and edge attributes containing the normalized edge weight.
+   *
+   * @see [[org.apache.spark.graphx.lib.PageRank]], method `runUntilConvergence`.
+   */
+  def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = {
+    PageRank.runUntilConvergence(graph, tol, resetProb)
+  }
+
+  /**
+   * Run PageRank for a fixed number of iterations returning a graph with vertex attributes
+   * containing the PageRank and edge attributes the normalized edge weight.
+   *
+   * @see [[org.apache.spark.graphx.lib.PageRank]], method `run`.
+   */
+  def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = {
+    PageRank.run(graph, numIter, resetProb)
+  }
+
+  /**
+   * Compute the connected component membership of each vertex and return a graph with the vertex
+   * value containing the lowest vertex id in the connected component containing that vertex.
+   *
+   * @see [[org.apache.spark.graphx.lib.ConnectedComponents]]
+   */
+  def connectedComponents(): Graph[VertexID, ED] = {
+    ConnectedComponents.run(graph)
+  }
+
+  /**
+   * Compute the number of triangles passing through each vertex.
+   *
+   * @see [[org.apache.spark.graphx.lib.TriangleCount]]
+   */
+  def triangleCount(): Graph[Int, ED] = {
+    TriangleCount.run(graph)
+  }
+
+  /**
+   * Compute the strongly connected component (SCC) of each vertex and return a graph with the
+   * vertex value containing the lowest vertex id in the SCC containing that vertex.
+   *
+   * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents]]
+   */
+  def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = {
+    StronglyConnectedComponents.run(graph, numIter)
+  }
 } // end of GraphOps

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d691e9f4/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala
deleted file mode 100644
index cbcd9c2..0000000
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala
+++ /dev/null
@@ -1,66 +0,0 @@
-package org.apache.spark.graphx.lib
-
-import scala.reflect.ClassTag
-
-import org.apache.spark.graphx._
-
-/**
- * Provides graph algorithms directly on [[org.apache.spark.graphx.Graph]] via an implicit
- * conversion.
- * @example
- * {{{
- * import org.apache.spark.graph.lib._
- * val graph: Graph[_, _] = loadGraph()
- * graph.connectedComponents()
- * }}}
- */
-class Algorithms[VD: ClassTag, ED: ClassTag](self: Graph[VD, ED]) {
-  /**
-   * Run a dynamic version of PageRank returning a graph with vertex attributes containing the
-   * PageRank and edge attributes containing the normalized edge weight.
-   *
-   * @see [[org.apache.spark.graphx.lib.PageRank]], method `runUntilConvergence`.
-   */
-  def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = {
-    PageRank.runUntilConvergence(self, tol, resetProb)
-  }
-
-  /**
-   * Run PageRank for a fixed number of iterations returning a graph with vertex attributes
-   * containing the PageRank and edge attributes the normalized edge weight.
-   *
-   * @see [[org.apache.spark.graphx.lib.PageRank]], method `run`.
-   */
-  def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = {
-    PageRank.run(self, numIter, resetProb)
-  }
-
-  /**
-   * Compute the connected component membership of each vertex and return a graph with the vertex
-   * value containing the lowest vertex id in the connected component containing that vertex.
-   *
-   * @see [[org.apache.spark.graphx.lib.ConnectedComponents]]
-   */
-  def connectedComponents(): Graph[VertexID, ED] = {
-    ConnectedComponents.run(self)
-  }
-
-  /**
-   * Compute the number of triangles passing through each vertex.
-   *
-   * @see [[org.apache.spark.graphx.lib.TriangleCount]]
-   */
-  def triangleCount(): Graph[Int, ED] = {
-    TriangleCount.run(self)
-  }
-
-  /**
-   * Compute the strongly connected component (SCC) of each vertex and return a graph with the
-   * vertex value containing the lowest vertex id in the SCC containing that vertex.
-   *
-   * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents]]
-   */
-  def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = {
-    StronglyConnectedComponents.run(self, numIter)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d691e9f4/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala
deleted file mode 100644
index f6f2626..0000000
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala
+++ /dev/null
@@ -1,8 +0,0 @@
-package org.apache.spark.graphx
-
-import scala.reflect.ClassTag
-
-package object lib {
-  implicit def graphToAlgorithms[VD: ClassTag, ED: ClassTag](
-      graph: Graph[VD, ED]): Algorithms[VD, ED] = new Algorithms(graph)
-}


[31/50] git commit: Moved SVDPlusPlusConf into SVDPlusPlus object itself.

Posted by pw...@apache.org.
Moved SVDPlusPlusConf into SVDPlusPlus object itself.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/8e5c7324
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/8e5c7324
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/8e5c7324

Branch: refs/heads/master
Commit: 8e5c7324303ee9a9a61ad35e94ada5638ca0cf70
Parents: 1dce9ce
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 18:45:20 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 18:45:20 2014 -0800

----------------------------------------------------------------------
 .../apache/spark/graphx/lib/SVDPlusPlus.scala   | 30 +++++++++++---------
 .../spark/graphx/lib/SVDPlusPlusSuite.scala     |  2 +-
 2 files changed, 17 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8e5c7324/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
index fa6b1db..ba6517e 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
@@ -5,19 +5,21 @@ import org.apache.commons.math.linear._
 import org.apache.spark.rdd._
 import org.apache.spark.graphx._
 
-/** Configuration parameters for SVDPlusPlus. */
-class SVDPlusPlusConf(
-  var rank: Int,
-  var maxIters: Int,
-  var minVal: Double,
-  var maxVal: Double,
-  var gamma1: Double,
-  var gamma2: Double,
-  var gamma6: Double,
-  var gamma7: Double) extends Serializable
-
 /** Implementation of SVD++ algorithm. */
 object SVDPlusPlus {
+
+  /** Configuration parameters for SVDPlusPlus. */
+  class Conf(
+      var rank: Int,
+      var maxIters: Int,
+      var minVal: Double,
+      var maxVal: Double,
+      var gamma1: Double,
+      var gamma2: Double,
+      var gamma6: Double,
+      var gamma7: Double)
+    extends Serializable
+
   /**
    * Implement SVD++ based on "Factorization Meets the Neighborhood:
    * a Multifaceted Collaborative Filtering Model",
@@ -32,7 +34,7 @@ object SVDPlusPlus {
    *
    * @return a graph with vertex attributes containing the trained model
    */
-  def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf)
+  def run(edges: RDD[Edge[Double]], conf: Conf)
     : (Graph[(RealVector, RealVector, Double, Double), Double], Double) =
   {
     // Generate default vertex attribute
@@ -64,7 +66,7 @@ object SVDPlusPlus {
         (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1))
     }
 
-    def mapTrainF(conf: SVDPlusPlusConf, u: Double)
+    def mapTrainF(conf: Conf, u: Double)
         (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double])
       : Iterator[(VertexID, (RealVector, RealVector, Double))] = {
       val (usr, itm) = (et.srcAttr, et.dstAttr)
@@ -112,7 +114,7 @@ object SVDPlusPlus {
     }
 
     // calculate error on training set
-    def mapTestF(conf: SVDPlusPlusConf, u: Double)
+    def mapTestF(conf: Conf, u: Double)
         (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double])
       : Iterator[(VertexID, Double)] =
     {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8e5c7324/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
index a4a1cda..e173c65 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
@@ -18,7 +18,7 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext {
         val fields = line.split(",")
         Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble)
       }
-      val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations
+      val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations
       var (graph, u) = SVDPlusPlus.run(edges, conf)
       graph.cache()
       val err = graph.vertices.collect.map{ case (vid, vd) =>


[39/50] git commit: Fix Pregel SSSP example in programming guide

Posted by pw...@apache.org.
Fix Pregel SSSP example in programming guide


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/59e4384e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/59e4384e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/59e4384e

Branch: refs/heads/master
Commit: 59e4384e19b0d7390259fa42daae95ae6f12f793
Parents: c6023be
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 21:02:09 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:02:38 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 22 ++++++++++++++--------
 1 file changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/59e4384e/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 76de26c..91cc5b6 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -511,7 +511,7 @@ In the following example we use the `mapReduceTriplets` operator to compute the
 more senior followers of each user.
 
 {% highlight scala %}
-// Import Random graph generation library
+// Import random graph generation library
 import org.apache.spark.graphx.util.GraphGenerators
 // Create a graph with "age" as the vertex property.  Here we use a random graph for simplicity.
 val graph: Graph[Double, Int] =
@@ -643,18 +643,23 @@ iterations, and the edge direction in which to send messages (by default along o
 second argument list contains the user defined functions for receiving messages (the vertex program
 `vprog`), computing messages (`sendMsg`), and combining messages `mergeMsg`.
 
-We can use the Pregel operator to express computation such single source shortest path in the
-following example.
+We can use the Pregel operator to express computation such as single source
+shortest path in the following example.
 
 {% highlight scala %}
-val graph: Graph[String, Double] // A graph with edge attributes containing distances
-val sourceId: VertexId = 42 // The ultimate source
+import org.apache.spark.graphx._
+// Import random graph generation library
+import org.apache.spark.graphx.util.GraphGenerators
+// A graph with edge attributes containing distances
+val graph: Graph[Int, Double] =
+  GraphGenerators.logNormalGraph(sc, numVertices = 100).mapEdges(e => e.attr.toDouble)
+val sourceId: VertexID = 42 // The ultimate source
 // Initialize the graph such that all vertices except the root have distance infinity.
-val initialGraph = graph.mapVertices((id, _) => if (id == shourceId) 0.0 else Double.PositiveInfinity)
+val initialGraph = graph.mapVertices((id, _) => if (id == sourceId) 0.0 else Double.PositiveInfinity)
 val sssp = initialGraph.pregel(Double.PositiveInfinity)(
-  (id, dist, newDist) => math.min(dist, newDist) // Vertex Program
+  (id, dist, newDist) => math.min(dist, newDist), // Vertex Program
   triplet => {  // Send Message
-    if(triplet.srcAttr + triplet.attr < triplet.dstAttr) {
+    if (triplet.srcAttr + triplet.attr < triplet.dstAttr) {
       Iterator((triplet.dstId, triplet.srcAttr + triplet.attr))
     } else {
       Iterator.empty
@@ -662,6 +667,7 @@ val sssp = initialGraph.pregel(Double.PositiveInfinity)(
   },
   (a,b) => math.min(a,b) // Merge Message
   )
+println(sssp.vertices.collect.mkString("\n"))
 {% endhighlight %}
 
 # Graph Builders


[04/50] git commit: Add PageRank example and data

Posted by pw...@apache.org.
Add PageRank example and data


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/5e35d39e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/5e35d39e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/5e35d39e

Branch: refs/heads/master
Commit: 5e35d39e0f26db3b669bc2318bd7b3f9f6c5fc50
Parents: f096f4e
Author: Ankur Dave <an...@gmail.com>
Authored: Sun Jan 12 13:10:53 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Sun Jan 12 13:10:53 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 32 +++++++++++++++++++-
 graphx/data/followers.txt                       | 12 ++++++++
 graphx/data/users.txt                           |  6 ++++
 .../org/apache/spark/graphx/lib/PageRank.scala  |  2 +-
 4 files changed, 50 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5e35d39e/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 7f93754..52668b0 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -470,10 +470,40 @@ things to worry about.)
 # Graph Algorithms
 <a name="graph_algorithms"></a>
 
-This section should describe the various algorithms and how they are used.
+GraphX includes a set of graph algorithms in to simplify analytics. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via an implicit conversion to [`Algorithms`][Algorithms]. This section describes the algorithms and how they are used.
+
+[Algorithms]: api/graphx/index.html#org.apache.spark.graphx.lib.Algorithms
 
 ## PageRank
 
+PageRank measures the importance of each vertex in a graph, assuming an edge from *u* to *v* represents an endorsement of *v*'s importance by *u*. For example, if a Twitter user is followed by many others, the user will be ranked highly.
+
+Spark includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We can compute the PageRank of each user as follows:
+
+{% highlight scala %}
+// Load the implicit conversion to Algorithms
+import org.apache.spark.graphx.lib._
+// Load the datasets into a graph
+val users = sc.textFile("graphx/data/users.txt").map { line =>
+  val fields = line.split("\\s+")
+  (fields(0).toLong, fields(1))
+}
+val followers = sc.textFile("graphx/data/followers.txt").map { line =>
+  val fields = line.split("\\s+")
+  Edge(fields(0).toLong, fields(1).toLong, 1)
+}
+val graph = Graph(users, followers)
+// Run PageRank
+val ranks = graph.pageRank(0.0001).vertices
+// Join the ranks with the usernames
+val ranksByUsername = users.leftOuterJoin(ranks).map {
+  case (id, (username, rankOpt)) => (username, rankOpt.getOrElse(0.0))
+}
+// Print the result
+println(ranksByUsername.collect().mkString("\n"))
+{% endhighlight %}
+
+
 ## Connected Components
 
 ## Shortest Path

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5e35d39e/graphx/data/followers.txt
----------------------------------------------------------------------
diff --git a/graphx/data/followers.txt b/graphx/data/followers.txt
new file mode 100644
index 0000000..0f46d80
--- /dev/null
+++ b/graphx/data/followers.txt
@@ -0,0 +1,12 @@
+2 1
+3 1
+4 1
+6 1
+3 2
+6 2
+7 2
+6 3
+7 3
+7 6
+6 7
+3 7

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5e35d39e/graphx/data/users.txt
----------------------------------------------------------------------
diff --git a/graphx/data/users.txt b/graphx/data/users.txt
new file mode 100644
index 0000000..ce3d06c
--- /dev/null
+++ b/graphx/data/users.txt
@@ -0,0 +1,6 @@
+1 BarackObama
+2 ericschmidt
+3 jeresig
+4 justinbieber
+6 matei_zaharia
+7 odersky

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5e35d39e/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
index 809b6d0..cf95267 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
@@ -106,7 +106,7 @@ object PageRank extends Logging {
    * @tparam ED the original edge attribute (not used)
    *
    * @param graph the graph on which to compute PageRank
-   * @param tol the tolerance allowed at convergence (smaller => more * accurate).
+   * @param tol the tolerance allowed at convergence (smaller => more accurate).
    * @param resetProb the random reset probability (alpha)
    *
    * @return the graph containing with each vertex containing the PageRank and each edge


[19/50] git commit: Merge branch 'master' into graphx

Posted by pw...@apache.org.
Merge branch 'master' into graphx


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/e2d25d2d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/e2d25d2d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/e2d25d2d

Branch: refs/heads/master
Commit: e2d25d2dfeb1d43d1e36f169250d8efef4ac232a
Parents: 8038da2 b93f9d4
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 16:21:26 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 16:21:26 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/bagel/BagelSuite.scala     |   1 -
 bin/pyspark                                     |   7 +-
 conf/log4j.properties.template                  |   5 +-
 core/pom.xml                                    |  10 +
 .../org/apache/spark/log4j-defaults.properties  |   5 +-
 .../scala/org/apache/spark/Accumulators.scala   |   2 +-
 .../scala/org/apache/spark/Aggregator.scala     |  61 +-
 .../scala/org/apache/spark/CacheManager.scala   |   4 +-
 .../scala/org/apache/spark/HttpFileServer.scala |   6 +-
 .../main/scala/org/apache/spark/Logging.scala   |   2 +-
 .../org/apache/spark/MapOutputTracker.scala     |  12 +-
 .../scala/org/apache/spark/Partitioner.scala    |  11 +-
 .../scala/org/apache/spark/SparkContext.scala   | 169 +++--
 .../main/scala/org/apache/spark/SparkEnv.scala  |  18 +-
 .../org/apache/spark/SparkHadoopWriter.scala    |  15 +-
 .../apache/spark/api/java/JavaDoubleRDD.scala   |  12 +-
 .../org/apache/spark/api/java/JavaPairRDD.scala |   6 +
 .../org/apache/spark/api/java/JavaRDD.scala     |   6 +
 .../org/apache/spark/api/java/JavaRDDLike.scala |   6 +
 .../spark/api/java/JavaSparkContext.scala       |  54 +-
 .../org/apache/spark/api/python/PythonRDD.scala |   7 +-
 .../apache/spark/broadcast/HttpBroadcast.scala  |   4 +-
 .../spark/broadcast/TorrentBroadcast.scala      |   8 +-
 .../scala/org/apache/spark/deploy/Client.scala  | 151 ++++
 .../apache/spark/deploy/ClientArguments.scala   | 117 +++
 .../org/apache/spark/deploy/DeployMessage.scala |  52 +-
 .../apache/spark/deploy/DriverDescription.scala |  29 +
 .../apache/spark/deploy/client/AppClient.scala  | 201 +++++
 .../spark/deploy/client/AppClientListener.scala |  39 +
 .../org/apache/spark/deploy/client/Client.scala | 200 -----
 .../spark/deploy/client/ClientListener.scala    |  39 -
 .../apache/spark/deploy/client/TestClient.scala |   4 +-
 .../apache/spark/deploy/master/DriverInfo.scala |  36 +
 .../spark/deploy/master/DriverState.scala       |  33 +
 .../master/FileSystemPersistenceEngine.scala    |  17 +-
 .../org/apache/spark/deploy/master/Master.scala | 195 ++++-
 .../spark/deploy/master/PersistenceEngine.scala |  11 +-
 .../apache/spark/deploy/master/WorkerInfo.scala |  20 +-
 .../master/ZooKeeperPersistenceEngine.scala     |  14 +-
 .../deploy/master/ui/ApplicationPage.scala      |   4 +-
 .../spark/deploy/master/ui/IndexPage.scala      |  56 +-
 .../spark/deploy/worker/CommandUtils.scala      |  63 ++
 .../spark/deploy/worker/DriverRunner.scala      | 234 ++++++
 .../spark/deploy/worker/DriverWrapper.scala     |  31 +
 .../spark/deploy/worker/ExecutorRunner.scala    |  67 +-
 .../org/apache/spark/deploy/worker/Worker.scala |  65 +-
 .../spark/deploy/worker/WorkerWatcher.scala     |  55 ++
 .../spark/deploy/worker/ui/IndexPage.scala      |  65 +-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  43 +-
 .../executor/CoarseGrainedExecutorBackend.scala |  28 +-
 .../org/apache/spark/executor/Executor.scala    |   7 +-
 .../org/apache/spark/io/CompressionCodec.scala  |   2 +-
 .../apache/spark/network/BufferMessage.scala    |   2 +-
 .../org/apache/spark/network/Connection.scala   |   6 +-
 .../spark/network/ConnectionManager.scala       |  18 +-
 .../org/apache/spark/network/Message.scala      |   6 +-
 .../spark/network/netty/ShuffleCopier.scala     |   2 +-
 .../spark/network/netty/ShuffleSender.scala     |   2 +-
 .../org/apache/spark/rdd/CheckpointRDD.scala    |   4 +-
 .../org/apache/spark/rdd/CoGroupedRDD.scala     |  88 ++-
 .../org/apache/spark/rdd/CoalescedRDD.scala     |  10 +-
 .../scala/org/apache/spark/rdd/HadoopRDD.scala  |  45 +-
 .../org/apache/spark/rdd/NewHadoopRDD.scala     |  36 +-
 .../org/apache/spark/rdd/PairRDDFunctions.scala |  59 +-
 .../scala/org/apache/spark/rdd/PipedRDD.scala   |   2 +-
 .../main/scala/org/apache/spark/rdd/RDD.scala   |  36 +-
 .../apache/spark/scheduler/DAGScheduler.scala   |   8 +-
 .../spark/scheduler/InputFormatInfo.scala       |   8 +-
 .../scala/org/apache/spark/scheduler/Pool.scala |   8 +-
 .../spark/scheduler/SchedulingAlgorithm.scala   |  11 +-
 .../apache/spark/scheduler/SparkListener.scala  |   3 +
 .../spark/scheduler/SparkListenerBus.scala      |  17 +-
 .../org/apache/spark/scheduler/Stage.scala      |   2 +-
 .../org/apache/spark/scheduler/TaskResult.scala |   2 +-
 .../spark/scheduler/TaskResultGetter.scala      |   4 +-
 .../spark/scheduler/TaskSchedulerImpl.scala     |   8 +-
 .../apache/spark/scheduler/TaskSetManager.scala |  22 +-
 .../cluster/CoarseGrainedSchedulerBackend.scala |   7 +-
 .../cluster/SimrSchedulerBackend.scala          |   2 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |  10 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |   4 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |   8 +-
 .../spark/serializer/KryoSerializer.scala       |   4 +-
 .../spark/storage/BlockFetcherIterator.scala    |   2 +-
 .../org/apache/spark/storage/BlockId.scala      |  12 +-
 .../org/apache/spark/storage/BlockManager.scala |  28 +-
 .../spark/storage/BlockManagerMaster.scala      |   4 +-
 .../spark/storage/BlockManagerMasterActor.scala |   7 +-
 .../spark/storage/BlockManagerWorker.scala      |  20 +-
 .../org/apache/spark/storage/BlockMessage.scala |   2 +-
 .../spark/storage/BlockMessageArray.scala       |   2 +-
 .../spark/storage/BlockObjectWriter.scala       |   4 +
 .../apache/spark/storage/DiskBlockManager.scala |  13 +-
 .../org/apache/spark/storage/MemoryStore.scala  |   2 +-
 .../spark/storage/ShuffleBlockManager.scala     |   4 +-
 .../org/apache/spark/storage/StorageLevel.scala |   2 +-
 .../spark/ui/jobs/JobProgressListener.scala     |   2 +-
 .../org/apache/spark/ui/jobs/StagePage.scala    |   2 +-
 .../org/apache/spark/ui/jobs/StageTable.scala   |   2 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala |  20 +-
 .../org/apache/spark/util/AppendOnlyMap.scala   | 237 ------
 .../org/apache/spark/util/ClosureCleaner.scala  |  10 +-
 .../org/apache/spark/util/MetadataCleaner.scala |   2 +-
 .../org/apache/spark/util/SizeEstimator.scala   |  10 +-
 .../apache/spark/util/TimeStampedHashMap.scala  |  17 +-
 .../scala/org/apache/spark/util/Utils.scala     |  64 +-
 .../scala/org/apache/spark/util/Vector.scala    |  20 +-
 .../org/apache/spark/util/XORShiftRandom.scala  |   2 +-
 .../spark/util/collection/AppendOnlyMap.scala   | 297 ++++++++
 .../util/collection/ExternalAppendOnlyMap.scala | 350 +++++++++
 .../collection/SizeTrackingAppendOnlyMap.scala  | 101 +++
 .../org/apache/spark/LocalSparkContext.scala    |   1 -
 .../apache/spark/MapOutputTrackerSuite.scala    |   1 -
 .../apache/spark/deploy/JsonProtocolSuite.scala |  40 +-
 .../spark/deploy/worker/DriverRunnerTest.scala  | 131 ++++
 .../deploy/worker/ExecutorRunnerTest.scala      |   4 +-
 .../deploy/worker/WorkerWatcherSuite.scala      |  32 +
 .../spark/scheduler/ClusterSchedulerSuite.scala |   9 +-
 .../spark/scheduler/DAGSchedulerSuite.scala     |   3 +-
 .../apache/spark/scheduler/JobLoggerSuite.scala |   7 +-
 .../spark/scheduler/TaskSetManagerSuite.scala   |   2 +-
 .../spark/storage/BlockManagerSuite.scala       |   4 -
 .../apache/spark/util/AppendOnlyMapSuite.scala  | 154 ----
 .../apache/spark/util/ClosureCleanerSuite.scala |  14 +-
 .../util/SizeTrackingAppendOnlyMapSuite.scala   | 120 +++
 .../org/apache/spark/util/VectorSuite.scala     |  44 ++
 .../apache/spark/util/XORShiftRandomSuite.scala |   2 +-
 .../util/collection/AppendOnlyMapSuite.scala    | 198 +++++
 .../collection/ExternalAppendOnlyMapSuite.scala | 230 ++++++
 docs/configuration.md                           |  25 +-
 docs/python-programming-guide.md                |   5 +-
 docs/running-on-yarn.md                         |  15 +-
 docs/spark-standalone.md                        |  38 +-
 docs/streaming-programming-guide.md             |   6 +-
 ec2/spark_ec2.py                                |  12 +-
 .../streaming/examples/JavaFlumeEventCount.java |   2 +
 .../streaming/examples/JavaKafkaWordCount.java  |   2 +
 .../examples/JavaNetworkWordCount.java          |  11 +-
 .../streaming/examples/JavaQueueStream.java     |   2 +
 .../spark/examples/DriverSubmissionTest.scala   |  46 ++
 .../org/apache/spark/examples/LocalALS.scala    |   8 +-
 .../org/apache/spark/examples/LocalFileLR.scala |   2 +-
 .../org/apache/spark/examples/LocalKMeans.scala |   2 +-
 .../org/apache/spark/examples/SparkALS.scala    |   6 +-
 .../org/apache/spark/examples/SparkHdfsLR.scala |   2 +-
 .../org/apache/spark/examples/SparkKMeans.scala |  12 +-
 .../streaming/examples/ActorWordCount.scala     |  12 +-
 .../streaming/examples/FlumeEventCount.scala    |   4 +-
 .../streaming/examples/HdfsWordCount.scala      |   3 +-
 .../streaming/examples/KafkaWordCount.scala     |   5 +-
 .../streaming/examples/MQTTWordCount.scala      |   8 +-
 .../streaming/examples/NetworkWordCount.scala   |   7 +-
 .../spark/streaming/examples/QueueStream.scala  |   8 +-
 .../streaming/examples/RawNetworkGrep.scala     |   7 +-
 .../examples/RecoverableNetworkWordCount.scala  | 118 +++
 .../examples/StatefulNetworkWordCount.scala     |   2 +
 .../streaming/examples/StreamingExamples.scala  |  21 +
 .../streaming/examples/TwitterAlgebirdCMS.scala |  14 +-
 .../streaming/examples/TwitterAlgebirdHLL.scala |  13 +-
 .../streaming/examples/TwitterPopularTags.scala |   6 +-
 .../streaming/examples/ZeroMQWordCount.scala    |   1 +
 .../clickstream/PageViewGenerator.scala         |   2 +-
 .../examples/clickstream/PageViewStream.scala   |   6 +-
 .../spark/streaming/flume/FlumeUtils.scala      |   4 +-
 external/kafka/pom.xml                          |   4 +-
 .../spark/streaming/kafka/KafkaUtils.scala      |   6 +-
 .../spark/streaming/mqtt/MQTTInputDStream.scala |   2 +-
 .../apache/spark/streaming/mqtt/MQTTUtils.scala |   4 +-
 .../spark/streaming/mqtt/MQTTStreamSuite.scala  |   2 +-
 .../spark/streaming/twitter/TwitterUtils.scala  |   7 +-
 .../streaming/twitter/TwitterStreamSuite.scala  |   2 +-
 .../spark/streaming/zeromq/ZeroMQUtils.scala    |   3 +-
 .../spark/mllib/api/python/PythonMLLibAPI.scala |  29 +-
 .../apache/spark/mllib/recommendation/ALS.scala |  15 +-
 pom.xml                                         |  17 +
 project/SparkBuild.scala                        |   3 +-
 .../scala/org/apache/spark/repl/ReplSuite.scala |   2 -
 sbt/sbt                                         |  51 +-
 .../org/apache/spark/streaming/Checkpoint.scala | 192 +++--
 .../apache/spark/streaming/ContextWaiter.scala  |  28 +
 .../org/apache/spark/streaming/DStream.scala    | 736 ------------------
 .../spark/streaming/DStreamCheckpointData.scala | 112 ---
 .../apache/spark/streaming/DStreamGraph.scala   |  50 +-
 .../spark/streaming/PairDStreamFunctions.scala  | 621 ---------------
 .../spark/streaming/StreamingContext.scala      | 180 +++--
 .../spark/streaming/api/java/JavaDStream.scala  |   3 +-
 .../streaming/api/java/JavaDStreamLike.scala    |  27 +-
 .../streaming/api/java/JavaPairDStream.scala    |   1 +
 .../api/java/JavaStreamingContext.scala         | 120 ++-
 .../spark/streaming/dstream/DStream.scala       | 757 +++++++++++++++++++
 .../dstream/DStreamCheckpointData.scala         | 126 +++
 .../streaming/dstream/FileInputDStream.scala    |  46 +-
 .../streaming/dstream/FilteredDStream.scala     |   2 +-
 .../dstream/FlatMapValuedDStream.scala          |   2 +-
 .../streaming/dstream/FlatMappedDStream.scala   |   2 +-
 .../streaming/dstream/ForEachDStream.scala      |   2 +-
 .../streaming/dstream/GlommedDStream.scala      |   2 +-
 .../spark/streaming/dstream/InputDStream.scala  |   4 +-
 .../dstream/MapPartitionedDStream.scala         |   2 +-
 .../streaming/dstream/MapValuedDStream.scala    |   2 +-
 .../spark/streaming/dstream/MappedDStream.scala |   2 +-
 .../streaming/dstream/NetworkInputDStream.scala |   6 +-
 .../dstream/PairDStreamFunctions.scala          | 622 +++++++++++++++
 .../dstream/ReducedWindowedDStream.scala        |   2 +-
 .../streaming/dstream/ShuffledDStream.scala     |   2 +-
 .../spark/streaming/dstream/StateDStream.scala  |  10 +-
 .../streaming/dstream/TransformedDStream.scala  |   2 +-
 .../spark/streaming/dstream/UnionDStream.scala  |   3 +-
 .../streaming/dstream/WindowedDStream.scala     |  17 +-
 .../apache/spark/streaming/scheduler/Job.scala  |  11 +-
 .../streaming/scheduler/JobGenerator.scala      | 102 ++-
 .../streaming/scheduler/JobScheduler.scala      | 141 ++--
 .../spark/streaming/scheduler/JobSet.scala      |  18 +-
 .../scheduler/NetworkInputTracker.scala         |  40 +-
 .../streaming/scheduler/StreamingListener.scala |   3 +-
 .../scheduler/StreamingListenerBus.scala        |  23 +-
 .../org/apache/spark/streaming/util/Clock.scala |   4 +-
 .../streaming/util/MasterFailureTest.scala      |  59 +-
 .../spark/streaming/util/RawTextHelper.scala    |   2 +-
 .../spark/streaming/util/RecurringTimer.scala   |  13 +-
 .../apache/spark/streaming/JavaAPISuite.java    |  29 +-
 .../streaming/LocalJavaStreamingContext.java    |   2 -
 .../spark/streaming/BasicOperationsSuite.scala  |   9 +-
 .../spark/streaming/CheckpointSuite.scala       |  35 +-
 .../spark/streaming/StreamingContextSuite.scala | 219 ++++++
 .../streaming/StreamingListenerSuite.scala      |   1 +
 .../apache/spark/streaming/TestSuiteBase.scala  |  10 +-
 .../spark/streaming/WindowOperationsSuite.scala |  15 +
 .../tools/JavaAPICompletenessChecker.scala      |  50 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  13 +-
 .../org/apache/spark/deploy/yarn/Client.scala   |   7 +-
 .../spark/deploy/yarn/WorkerLauncher.scala      |  36 +-
 .../spark/deploy/yarn/WorkerRunnable.scala      |   7 +-
 .../yarn/ClientDistributedCacheManager.scala    |  10 +-
 .../cluster/YarnClientSchedulerBackend.scala    |  50 +-
 .../ClientDistributedCacheManagerSuite.scala    |   2 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  13 +-
 .../org/apache/spark/deploy/yarn/Client.scala   |   4 +-
 .../spark/deploy/yarn/WorkerLauncher.scala      |  28 +-
 239 files changed, 6896 insertions(+), 3314 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2d25d2d/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2d25d2d/core/src/main/scala/org/apache/spark/rdd/RDD.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2d25d2d/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e2d25d2d/project/SparkBuild.scala
----------------------------------------------------------------------


[28/50] git commit: Moved PartitionStrategy's into an object.

Posted by pw...@apache.org.
Moved PartitionStrategy's into an object.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1dce9ce4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1dce9ce4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1dce9ce4

Branch: refs/heads/master
Commit: 1dce9ce446dd248755cd65b7a6a0729a4dca2d62
Parents: ae06d2c
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 18:32:04 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 18:32:04 2014 -0800

----------------------------------------------------------------------
 .../apache/spark/graphx/PartitionStrategy.scala | 158 ++++++++++---------
 .../org/apache/spark/graphx/lib/Analytics.scala |   2 +
 .../org/apache/spark/graphx/GraphSuite.scala    |   1 +
 .../spark/graphx/lib/TriangleCountSuite.scala   |   5 +-
 4 files changed, 85 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1dce9ce4/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
index b9ccd87..6d2990a 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
@@ -4,96 +4,100 @@ package org.apache.spark.graphx
  * Represents the way edges are assigned to edge partitions based on their source and destination
  * vertex IDs.
  */
-sealed trait PartitionStrategy extends Serializable {
+trait PartitionStrategy extends Serializable {
   /** Returns the partition number for a given edge. */
   def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID
 }
 
-
 /**
- * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix,
- * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication.
- *
- * Suppose we have a graph with 11 vertices that we want to partition
- * over 9 machines.  We can use the following sparse matrix representation:
- *
- * <pre>
- *       __________________________________
- *  v0   | P0 *     | P1       | P2    *  |
- *  v1   |  ****    |  *       |          |
- *  v2   |  ******* |      **  |  ****    |
- *  v3   |  *****   |  *  *    |       *  |
- *       ----------------------------------
- *  v4   | P3 *     | P4 ***   | P5 **  * |
- *  v5   |  *  *    |  *       |          |
- *  v6   |       *  |      **  |  ****    |
- *  v7   |  * * *   |  *  *    |       *  |
- *       ----------------------------------
- *  v8   | P6   *   | P7    *  | P8  *   *|
- *  v9   |     *    |  *    *  |          |
- *  v10  |       *  |      **  |  *  *    |
- *  v11  | * <-E    |  ***     |       ** |
- *       ----------------------------------
- * </pre>
- *
- * The edge denoted by `E` connects `v11` with `v1` and is assigned to processor `P6`. To get the
- * processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks.  Notice
- * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, P6)` or the last
- * row of blocks `(P6, P7, P8)`.  As a consequence we can guarantee that `v11` will need to be
- * replicated to at most `2 * sqrt(numParts)` machines.
- *
- * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work
- * balance.  To improve balance we first multiply each vertex id by a large prime to shuffle the
- * vertex locations.
- *
- * One of the limitations of this approach is that the number of machines must either be a perfect
- * square. We partially address this limitation by computing the machine assignment to the next
- * largest perfect square and then mapping back down to the actual number of machines.
- * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect square
- * is used.
+ * Collection of built-in [[PartitionStrategy]] implementations.
  */
-case object EdgePartition2D extends PartitionStrategy {
-  override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
-    val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt
-    val mixingPrime: VertexID = 1125899906842597L
-    val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt
-    val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt
-    (col * ceilSqrtNumParts + row) % numParts
+object PartitionStrategy {
+  /**
+   * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix,
+   * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication.
+   *
+   * Suppose we have a graph with 11 vertices that we want to partition
+   * over 9 machines.  We can use the following sparse matrix representation:
+   *
+   * <pre>
+   *       __________________________________
+   *  v0   | P0 *     | P1       | P2    *  |
+   *  v1   |  ****    |  *       |          |
+   *  v2   |  ******* |      **  |  ****    |
+   *  v3   |  *****   |  *  *    |       *  |
+   *       ----------------------------------
+   *  v4   | P3 *     | P4 ***   | P5 **  * |
+   *  v5   |  *  *    |  *       |          |
+   *  v6   |       *  |      **  |  ****    |
+   *  v7   |  * * *   |  *  *    |       *  |
+   *       ----------------------------------
+   *  v8   | P6   *   | P7    *  | P8  *   *|
+   *  v9   |     *    |  *    *  |          |
+   *  v10  |       *  |      **  |  *  *    |
+   *  v11  | * <-E    |  ***     |       ** |
+   *       ----------------------------------
+   * </pre>
+   *
+   * The edge denoted by `E` connects `v11` with `v1` and is assigned to processor `P6`. To get the
+   * processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks.  Notice
+   * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, P6)` or the last
+   * row of blocks `(P6, P7, P8)`.  As a consequence we can guarantee that `v11` will need to be
+   * replicated to at most `2 * sqrt(numParts)` machines.
+   *
+   * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work
+   * balance.  To improve balance we first multiply each vertex id by a large prime to shuffle the
+   * vertex locations.
+   *
+   * One of the limitations of this approach is that the number of machines must either be a perfect
+   * square. We partially address this limitation by computing the machine assignment to the next
+   * largest perfect square and then mapping back down to the actual number of machines.
+   * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect square
+   * is used.
+   */
+  case object EdgePartition2D extends PartitionStrategy {
+    override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
+      val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt
+      val mixingPrime: VertexID = 1125899906842597L
+      val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt
+      val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt
+      (col * ceilSqrtNumParts + row) % numParts
+    }
   }
-}
 
-/**
- * Assigns edges to partitions using only the source vertex ID, colocating edges with the same
- * source.
- */
-case object EdgePartition1D extends PartitionStrategy {
-  override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
-    val mixingPrime: VertexID = 1125899906842597L
-    (math.abs(src) * mixingPrime).toInt % numParts
+  /**
+   * Assigns edges to partitions using only the source vertex ID, colocating edges with the same
+   * source.
+   */
+  case object EdgePartition1D extends PartitionStrategy {
+    override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
+      val mixingPrime: VertexID = 1125899906842597L
+      (math.abs(src) * mixingPrime).toInt % numParts
+    }
   }
-}
 
 
-/**
- * Assigns edges to partitions by hashing the source and destination vertex IDs, resulting in a
- * random vertex cut that colocates all same-direction edges between two vertices.
- */
-case object RandomVertexCut extends PartitionStrategy {
-  override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
-    math.abs((src, dst).hashCode()) % numParts
+  /**
+   * Assigns edges to partitions by hashing the source and destination vertex IDs, resulting in a
+   * random vertex cut that colocates all same-direction edges between two vertices.
+   */
+  case object RandomVertexCut extends PartitionStrategy {
+    override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
+      math.abs((src, dst).hashCode()) % numParts
+    }
   }
-}
 
 
-/**
- * Assigns edges to partitions by hashing the source and destination vertex IDs in a canonical
- * direction, resulting in a random vertex cut that colocates all edges between two vertices,
- * regardless of direction.
- */
-case object CanonicalRandomVertexCut extends PartitionStrategy {
-  override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
-    val lower = math.min(src, dst)
-    val higher = math.max(src, dst)
-    math.abs((lower, higher).hashCode()) % numParts
+  /**
+   * Assigns edges to partitions by hashing the source and destination vertex IDs in a canonical
+   * direction, resulting in a random vertex cut that colocates all edges between two vertices,
+   * regardless of direction.
+   */
+  case object CanonicalRandomVertexCut extends PartitionStrategy {
+    override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
+      val lower = math.min(src, dst)
+      val higher = math.max(src, dst)
+      math.abs((lower, higher).hashCode()) % numParts
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1dce9ce4/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
index d5e1de1..e0aff56 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
@@ -2,6 +2,7 @@ package org.apache.spark.graphx.lib
 
 import org.apache.spark._
 import org.apache.spark.graphx._
+import org.apache.spark.graphx.PartitionStrategy._
 
 /**
  * Driver program for running graph algorithms.
@@ -20,6 +21,7 @@ object Analytics extends Logging {
     }
 
     def pickPartitioner(v: String): PartitionStrategy = {
+      // TODO: Use reflection rather than listing all the partitioning strategies here.
       v match {
         case "RandomVertexCut" => RandomVertexCut
         case "EdgePartition1D" => EdgePartition1D

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1dce9ce4/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
index c32a6cb..9587f04 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
@@ -4,6 +4,7 @@ import org.scalatest.FunSuite
 
 import org.apache.spark.SparkContext
 import org.apache.spark.graphx.Graph._
+import org.apache.spark.graphx.PartitionStrategy._
 import org.apache.spark.rdd._
 
 class GraphSuite extends FunSuite with LocalSparkContext {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1dce9ce4/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
index a286b7d..3452ce9 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
@@ -2,11 +2,8 @@ package org.apache.spark.graphx.lib
 
 import org.scalatest.FunSuite
 
-import org.apache.spark.SparkContext
-import org.apache.spark.SparkContext._
 import org.apache.spark.graphx._
-import org.apache.spark.graphx.util.GraphGenerators
-import org.apache.spark.rdd._
+import org.apache.spark.graphx.PartitionStrategy.RandomVertexCut
 
 
 class TriangleCountSuite extends FunSuite with LocalSparkContext {


[08/50] git commit: Add TriangleCount example

Posted by pw...@apache.org.
Add TriangleCount example


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/20c509b8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/20c509b8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/20c509b8

Branch: refs/heads/master
Commit: 20c509b805dbfd0ebb11d2d7bd53a4379249a86f
Parents: 2216319
Author: Ankur Dave <an...@gmail.com>
Authored: Sun Jan 12 21:41:21 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Sun Jan 12 21:41:32 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 31 +++++++++++++++++---
 .../apache/spark/graphx/lib/TriangleCount.scala |  5 ++--
 2 files changed, 29 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20c509b8/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 8975941..0e228d8 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -676,7 +676,9 @@ GraphX includes a set of graph algorithms in to simplify analytics. The algorith
 
 PageRank measures the importance of each vertex in a graph, assuming an edge from *u* to *v* represents an endorsement of *v*'s importance by *u*. For example, if a Twitter user is followed by many others, the user will be ranked highly.
 
-Spark includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We can compute the PageRank of each user as follows:
+GraphX comes with static and dynamic implementations of PageRank as methods on the [`PageRank` object][PageRank]. Static PageRank runs for a fixed number of iterations, while dynamic PageRank runs until the ranks converge (i.e., stop changing by more than a specified tolerance). GraphX also includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We compute the PageRank of each user as follows:
+
+[PageRank]: api/graphx/index.html#org.apache.spark.graphx.lib.PageRank$
 
 {% highlight scala %}
 // Load the implicit conversion to Algorithms
@@ -703,7 +705,9 @@ println(ranksByUsername.collect().mkString("\n"))
 
 ## Connected Components
 
-The connected components algorithm labels each connected component of the graph with the ID of its lowest-numbered vertex. For example, in a social network, connected components can approximate clusters. We can compute the connected components of the example social network dataset from the [PageRank section](#pagerank) as follows:
+The connected components algorithm labels each connected component of the graph with the ID of its lowest-numbered vertex. For example, in a social network, connected components can approximate clusters. GraphX contains an implementation of the algorithm in the [`ConnectedComponents` object][ConnectedComponents], and we compute the connected components of the example social network dataset from the [PageRank section](#pagerank) as follows:
+
+[ConnectedComponents]: api/graphx/index.html#org.apache.spark.graphx.lib.ConnectedComponents$
 
 {% highlight scala %}
 // Load the implicit conversion and graph as in the PageRank example
@@ -721,10 +725,29 @@ val ccByUsername = graph.vertices.innerJoin(cc) { (id, username, cc) =>
 println(ccByUsername.collect().mkString("\n"))
 {% endhighlight %}
 
-## Shortest Path
-
 ## Triangle Counting
 
+A vertex is part of a triangle when it has two adjacent vertices with an edge between them. GraphX implements a triangle counting algorithm in the [`TriangleCount` object][TriangleCount] that determines the number of triangles passing through each vertex, providing a measure of clustering. We compute the triangle count of the social network dataset from the [PageRank section](#pagerank). *Note that `TriangleCount` requires the edges to be in canonical orientation (`srcId < dstId`) and the graph to be partitioned using [`Graph#partitionBy`][Graph.partitionBy].*
+
+[TriangleCount]: api/graphx/index.html#org.apache.spark.graphx.lib.TriangleCount$
+[Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph@partitionBy(PartitionStrategy):Graph[VD,ED]
+
+{% highlight scala %}
+// Load the implicit conversion and graph as in the PageRank example
+import org.apache.spark.graphx.lib._
+val users = ...
+// Load the edges in canonical order and partition the graph for triangle count
+val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(RandomVertexCut)
+// Find the triangle count for each vertex
+val triCounts = graph.triangleCount().vertices
+// Join the triangle counts with the usernames
+val triCountByUsername = graph.vertices.innerJoin(triCounts) { (id, username, tc) =>
+  (username, tc)
+}
+// Print the result
+println(triCountByUsername.collect().mkString("\n"))
+{% endhighlight %}
+
 ## K-Core
 
 ## LDA

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/20c509b8/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
index c6b1c73..58da9e3 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
@@ -19,9 +19,8 @@ object TriangleCount {
    *
    *
    * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned
-   * using Graph.partitionBy.
-   *
-   * @return
+   * using [[org.apache.spark.graphx.Graph#partitionBy]], and its edges must be in canonical
+   * orientation (srcId < dstId).
    */
   def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = {
     // Remove redundant edges


[44/50] git commit: Edge object must be public for Edge case class

Posted by pw...@apache.org.
Edge object must be public for Edge case class


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c6dbfd16
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c6dbfd16
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c6dbfd16

Branch: refs/heads/master
Commit: c6dbfd16942f1dc0673071580ca968dd4f8428da
Parents: 6f6f8c9
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 22:08:44 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 22:08:44 2014 -0800

----------------------------------------------------------------------
 graphx/src/main/scala/org/apache/spark/graphx/Edge.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c6dbfd16/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
index 21be58e..738a38b 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
@@ -37,8 +37,8 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED]
     if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In }
 }
 
-private[graphx] object Edge {
-  def lexicographicOrdering[ED] = new Ordering[Edge[ED]] {
+object Edge {
+  private[graphx] def lexicographicOrdering[ED] = new Ordering[Edge[ED]] {
     override def compare(a: Edge[ED], b: Edge[ED]): Int =
       (if (a.srcId != b.srcId) a.srcId - b.srcId else a.dstId - b.dstId).toInt
   }


[29/50] git commit: Minor changes in graphx programming guide.

Posted by pw...@apache.org.
Minor changes in graphx programming guide.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/622b7f7d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/622b7f7d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/622b7f7d

Branch: refs/heads/master
Commit: 622b7f7d391375cced8633e4a2546dbca60a3907
Parents: 1dce9ce
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Mon Jan 13 17:46:47 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Mon Jan 13 18:40:43 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/622b7f7d/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index c82c3d7..c6505d2 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -543,7 +543,6 @@ val maxOutDegree: (VertexID, Int) = graph.outDegrees.reduce(max)
 val maxDegrees: (VertexID, Int)   = graph.degrees.reduce(max)
 {% endhighlight %}
 
-
 ### Collecting Neighbors
 
 In some cases it may be easier to express computation by collecting neighboring vertices and their
@@ -562,8 +561,8 @@ def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[ Array[(VertexID,
 # Pregel API
 <a name="pregel"></a>
 
-Graphs are inherently recursive data-structures as properties of a vertices depend on properties of
-their neighbors which intern depend on properties of the neighbors of their neighbors.  As a
+Graphs are inherently recursive data-structures as properties of vertices depend on properties of
+their neighbors which intern depend on properties of *their* neighbors.  As a
 consequence many important graph algorithms iteratively recompute the properties of each vertex
 until a fixed-point condition is reached.  A range of graph-parallel abstractions have been proposed
 to express these iterative algorithms.  GraphX exposes a Pregel operator which is a fusion of


[33/50] git commit: Updated doc for PageRank.

Posted by pw...@apache.org.
Updated doc for PageRank.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0b18bfba
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0b18bfba
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0b18bfba

Branch: refs/heads/master
Commit: 0b18bfba1aba60c2a1f576f10d9ab2fa316ebfa0
Parents: 9317286
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 18:51:04 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 18:51:04 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/graphx/lib/PageRank.scala  | 86 +++++++++-----------
 1 file changed, 39 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0b18bfba/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
index b205669..08256dc 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
@@ -5,7 +5,42 @@ import scala.reflect.ClassTag
 import org.apache.spark.Logging
 import org.apache.spark.graphx._
 
-/** PageRank algorithm implementation. */
+/**
+ * PageRank algorithm implementation. There are two implementations of PageRank implemented.
+ *
+ * The first implementation uses the [[Pregel]] interface and runs PageRank for a fixed number
+ * of iterations:
+ * {{{
+ * var PR = Array.fill(n)( 1.0 )
+ * val oldPR = Array.fill(n)( 1.0 )
+ * for( iter <- 0 until numIter ) {
+ *   swap(oldPR, PR)
+ *   for( i <- 0 until n ) {
+ *     PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum
+ *   }
+ * }
+ * }}}
+ *
+ * The second implementation uses the standalone [[Graph]] interface and runs PageRank until
+ * convergence:
+ *
+ * {{{
+ * var PR = Array.fill(n)( 1.0 )
+ * val oldPR = Array.fill(n)( 0.0 )
+ * while( max(abs(PR - oldPr)) > tol ) {
+ *   swap(oldPR, PR)
+ *   for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) {
+ *     PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum
+ *   }
+ * }
+ * }}}
+ *
+ * `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of
+ * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`.
+ *
+ * Note that this is not the "normalized" PageRank and as a consequence pages that have no
+ * inlinks will have a PageRank of alpha.
+ */
 object PageRank extends Logging {
 
   /**
@@ -13,26 +48,6 @@ object PageRank extends Logging {
    * with vertex attributes containing the PageRank and edge
    * attributes the normalized edge weight.
    *
-   * The following PageRank fixed point is computed for each vertex.
-   *
-   * {{{
-   * var PR = Array.fill(n)( 1.0 )
-   * val oldPR = Array.fill(n)( 1.0 )
-   * for( iter <- 0 until numIter ) {
-   *   swap(oldPR, PR)
-   *   for( i <- 0 until n ) {
-   *     PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum
-   *   }
-   * }
-   * }}}
-   *
-   * where `alpha` is the random reset probability (typically 0.15),
-   * `inNbrs[i]` is the set of neighbors whick link to `i` and
-   * `outDeg[j]` is the out degree of vertex `j`.
-   *
-   * Note that this is not the "normalized" PageRank and as a consequence pages that have no
-   * inlinks will have a PageRank of alpha.
-   *
    * @tparam VD the original vertex attribute (not used)
    * @tparam ED the original edge attribute (not used)
    *
@@ -47,16 +62,11 @@ object PageRank extends Logging {
   def run[VD: ClassTag, ED: ClassTag](
       graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] =
   {
-
-    /**
-     * Initialize the pagerankGraph with each edge attribute having
-     * weight 1/outDegree and each vertex with attribute 1.0.
-     */
+    // Initialize the pagerankGraph with each edge attribute having
+    // weight 1/outDegree and each vertex with attribute 1.0.
     val pagerankGraph: Graph[Double, Double] = graph
       // Associate the degree with each vertex
-      .outerJoinVertices(graph.outDegrees){
-      (vid, vdata, deg) => deg.getOrElse(0)
-    }
+      .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) }
       // Set the weight on the edges based on the degree
       .mapTriplets( e => 1.0 / e.srcAttr )
       // Set the vertex attributes to the initial pagerank values
@@ -85,23 +95,6 @@ object PageRank extends Logging {
    * Run a dynamic version of PageRank returning a graph with vertex attributes containing the
    * PageRank and edge attributes containing the normalized edge weight.
    *
-   * {{{
-   * var PR = Array.fill(n)( 1.0 )
-   * val oldPR = Array.fill(n)( 0.0 )
-   * while( max(abs(PR - oldPr)) > tol ) {
-   *   swap(oldPR, PR)
-   *   for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) {
-   *     PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum
-   *   }
-   * }
-   * }}}
-   *
-   * where `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of
-   * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`.
-   *
-   * Note that this is not the "normalized" PageRank and as a consequence pages that have no
-   * inlinks will have a PageRank of alpha.
-   *
    * @tparam VD the original vertex attribute (not used)
    * @tparam ED the original edge attribute (not used)
    *
@@ -157,5 +150,4 @@ object PageRank extends Logging {
       vertexProgram, sendMessage, messageCombiner)
       .mapVertices((vid, attr) => attr._1)
   } // end of deltaPageRank
-
 }


[18/50] git commit: Merge pull request #2 from jegonzal/GraphXCCIssue

Posted by pw...@apache.org.
Merge pull request #2 from jegonzal/GraphXCCIssue

Improving documentation and identifying potential bug in CC calculation.

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/8038da23
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/8038da23
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/8038da23

Branch: refs/heads/master
Commit: 8038da232870fe016e73122a2ef110ac8e56ca1e
Parents: 97cd27e 80e4d98
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 14:59:30 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 14:59:30 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 33 ++++++++++++---
 .../org/apache/spark/graphx/GraphOps.scala      |  4 +-
 .../spark/graphx/lib/ConnectedComponents.scala  | 44 +++++++++++++-------
 .../graphx/lib/ConnectedComponentsSuite.scala   | 30 +++++++++++++
 4 files changed, 89 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8038da23/docs/graphx-programming-guide.md
----------------------------------------------------------------------


[47/50] git commit: Fix all code examples in guide

Posted by pw...@apache.org.
Fix all code examples in guide


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/af645be5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/af645be5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/af645be5

Branch: refs/heads/master
Commit: af645be5b8d41d5a0fd4a529956c5ab438198db4
Parents: 2cd9358
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 22:29:45 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 22:29:45 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 46 +++++++++++++++++------------------
 graphx/data/users.txt            | 13 +++++-----
 2 files changed, 30 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af645be5/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 226299a..a7ab003 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -357,7 +357,7 @@ val relationships: RDD[Edge[String]] =
 val defaultUser = ("John Doe", "Missing")
 // Build the initial Graph
 val graph = Graph(users, relationships, defaultUser)
-// Notice that there is a user 0 (for which we have no information) connecting users
+// Notice that there is a user 0 (for which we have no information) connected to users
 // 4 (peter) and 5 (franklin).
 graph.triplets.map(
     triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1
@@ -858,11 +858,11 @@ val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt")
 val ranks = graph.pageRank(0.0001).vertices
 // Join the ranks with the usernames
 val users = sc.textFile("graphx/data/users.txt").map { line =>
-  val fields = line.split("\\s+")
+  val fields = line.split(",")
   (fields(0).toLong, fields(1))
 }
-val ranksByUsername = users.leftOuterJoin(ranks).map {
-  case (id, (username, rankOpt)) => (username, rankOpt.getOrElse(0.0))
+val ranksByUsername = users.join(ranks).map {
+  case (id, (username, rank)) => (username, rank)
 }
 // Print the result
 println(ranksByUsername.collect().mkString("\n"))
@@ -881,11 +881,11 @@ val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt")
 val cc = graph.connectedComponents().vertices
 // Join the connected components with the usernames
 val users = sc.textFile("graphx/data/users.txt").map { line =>
-  val fields = line.split("\\s+")
+  val fields = line.split(",")
   (fields(0).toLong, fields(1))
 }
-val ccByUsername = users.join(cc).map { case (id, (username, cc)) =>
-  (username, cc)
+val ccByUsername = users.join(cc).map {
+  case (id, (username, cc)) => (username, cc)
 }
 // Print the result
 println(ccByUsername.collect().mkString("\n"))
@@ -900,12 +900,12 @@ A vertex is part of a triangle when it has two adjacent vertices with an edge be
 
 {% highlight scala %}
 // Load the edges in canonical order and partition the graph for triangle count
-val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(RandomVertexCut)
+val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(PartitionStrategy.RandomVertexCut)
 // Find the triangle count for each vertex
 val triCounts = graph.triangleCount().vertices
 // Join the triangle counts with the usernames
 val users = sc.textFile("graphx/data/users.txt").map { line =>
-  val fields = line.split("\\s+")
+  val fields = line.split(",")
   (fields(0).toLong, fields(1))
 }
 val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) =>
@@ -934,32 +934,32 @@ all of this in just a few lines with GraphX:
 // Connect to the Spark cluster
 val sc = new SparkContext("spark://master.amplab.org", "research")
 
-// Load my user data and prase into tuples of user id and attribute list
-val users = sc.textFile("hdfs://user_attributes.tsv")
-  .map(line => line.split).map( parts => (parts.head, parts.tail) )
+// Load my user data and parse into tuples of user id and attribute list
+val users = (sc.textFile("graphx/data/users.txt")
+  .map(line => line.split(",")).map( parts => (parts.head.toLong, parts.tail) ))
 
 // Parse the edge data which is already in userId -> userId format
-val followerGraph = Graph.textFile(sc, "hdfs://followers.tsv")
+val followerGraph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt")
 
 // Attach the user attributes
-val graph = followerGraph.outerJoinVertices(users){
+val graph = followerGraph.outerJoinVertices(users) {
   case (uid, deg, Some(attrList)) => attrList
   // Some users may not have attributes so we set them as empty
   case (uid, deg, None) => Array.empty[String]
-  }
+}
 
-// Restrict the graph to users which have exactly two attributes
-val subgraph = graph.subgraph((vid, attr) => attr.size == 2)
+// Restrict the graph to users with usernames and names
+val subgraph = graph.subgraph(vpred = (vid, attr) => attr.size == 2)
 
 // Compute the PageRank
-val pagerankGraph = Analytics.pagerank(subgraph)
+val pagerankGraph = subgraph.pageRank(0.001)
 
 // Get the attributes of the top pagerank users
-val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices){
-  case (uid, attrList, Some(pr)) => (pr, attrList)
-  case (uid, attrList, None) => (pr, attrList)
-  }
+val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices) {
+  case (uid, attrList, Some(pr)) => (pr, attrList.toList)
+  case (uid, attrList, None) => (0.0, attrList.toList)
+}
 
-println(userInfoWithPageRank.top(5))
+println(userInfoWithPageRank.vertices.top(5)(Ordering.by(_._2._1)).mkString("\n"))
 
 {% endhighlight %}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af645be5/graphx/data/users.txt
----------------------------------------------------------------------
diff --git a/graphx/data/users.txt b/graphx/data/users.txt
index 26e3b3b..982d19d 100644
--- a/graphx/data/users.txt
+++ b/graphx/data/users.txt
@@ -1,6 +1,7 @@
-1 BarackObama
-2 ladygaga
-3 jeresig
-4 justinbieber
-6 matei_zaharia
-7 odersky
+1,BarackObama,Barack Obama
+2,ladygaga,Goddess of Love
+3,jeresig,John Resig
+4,justinbieber,Justin Bieber
+6,matei_zaharia,Matei Zaharia
+7,odersky,Martin Odersky
+8,anonsys


[26/50] git commit: Made more things private.

Posted by pw...@apache.org.
Made more things private.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/87f335db
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/87f335db
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/87f335db

Branch: refs/heads/master
Commit: 87f335db78221fc250bd64f39a334293db490379
Parents: a4e12af
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 18:30:26 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 18:30:26 2014 -0800

----------------------------------------------------------------------
 graphx/src/main/scala/org/apache/spark/graphx/Edge.scala |  8 ++++----
 .../main/scala/org/apache/spark/graphx/GraphOps.scala    |  4 ----
 .../org/apache/spark/graphx/impl/EdgePartition.scala     |  1 +
 .../apache/spark/graphx/impl/EdgePartitionBuilder.scala  |  1 +
 .../scala/org/apache/spark/graphx/impl/GraphImpl.scala   |  2 +-
 .../apache/spark/graphx/impl/MessageToPartition.scala    |  5 +++++
 .../apache/spark/graphx/impl/ReplicatedVertexView.scala  |  3 ++-
 .../org/apache/spark/graphx/impl/RoutingTable.scala      |  1 +
 .../scala/org/apache/spark/graphx/impl/Serializers.scala | 11 ++++++++++-
 .../scala/org/apache/spark/graphx/impl/package.scala     |  2 +-
 10 files changed, 26 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
index 6c396c3..8546305 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
@@ -1,18 +1,18 @@
 package org.apache.spark.graphx
 
-
 /**
  * A single directed edge consisting of a source id, target id,
  * and the data associated with the edge.
  *
  * @tparam ED type of the edge attribute
+ *
+ * @param srcId The vertex id of the source vertex
+ * @param dstId The vertex id of the target vertex
+ * @param attr The attribute associated with the edge
  */
 case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] (
-    /** The vertex id of the source vertex */
     var srcId: VertexID = 0,
-    /** The vertex id of the target vertex. */
     var dstId: VertexID = 0,
-    /** The attribute associated with the edge. */
     var attr: ED = null.asInstanceOf[ED])
   extends Serializable {
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 66d5180..447ef55 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -209,12 +209,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    * This function iterates until there are no remaining messages, or
    * for `maxIterations` iterations.
    *
-   * @tparam VD the vertex data type
-   * @tparam ED the edge data type
    * @tparam A the Pregel message type
    *
-   * @param graph the input graph.
-   *
    * @param initialMsg the message each vertex will receive at the on
    * the first iteration
    *

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
index d4d7162..ee95ead 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
@@ -15,6 +15,7 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap
  * @param index a clustered index on source vertex id
  * @tparam ED the edge attribute type.
  */
+private[graphx]
 class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag](
     val srcIds: Array[VertexID],
     val dstIds: Array[VertexID],

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
index fbc2940..9d072f9 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
@@ -7,6 +7,7 @@ import org.apache.spark.graphx._
 import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap
 import org.apache.spark.util.collection.PrimitiveVector
 
+private[graphx]
 class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) {
   var edges = new PrimitiveVector[Edge[ED]](size)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 97ca642..348490c 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -15,7 +15,7 @@ import org.apache.spark.util.ClosureCleaner
 
 
 /**
- * A Graph RDD that supports computation on graphs.
+ * A graph that supports computation on graphs.
  *
  * Graphs are represented using two classes of data: vertex-partitioned and
  * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges`

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala
index ad5daf8..05508ff 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala
@@ -7,6 +7,7 @@ import org.apache.spark.graphx.{PartitionID, VertexID}
 import org.apache.spark.rdd.{ShuffledRDD, RDD}
 
 
+private[graphx]
 class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T](
     @transient var partition: PartitionID,
     var vid: VertexID,
@@ -26,6 +27,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T](
  * @param partition index of the target partition.
  * @param data value to send
  */
+private[graphx]
 class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T](
     @transient var partition: PartitionID,
     var data: T)
@@ -39,6 +41,7 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef
 }
 
 
+private[graphx]
 class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) {
   def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = {
     val rdd = new ShuffledRDD[PartitionID, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner)
@@ -56,6 +59,7 @@ class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T
 }
 
 
+private[graphx]
 class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) {
 
   /**
@@ -68,6 +72,7 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) {
 }
 
 
+private[graphx]
 object MsgRDDFunctions {
   implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = {
     new MsgRDDFunctions(rdd)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
index 0e2f5a9..4ebe0b0 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
@@ -138,7 +138,7 @@ class ReplicatedVertexView[VD: ClassTag](
   }
 }
 
-object ReplicatedVertexView {
+private object ReplicatedVertexView {
   protected def buildBuffer[VD: ClassTag](
       pid2vidIter: Iterator[Array[Array[VertexID]]],
       vertexPartIter: Iterator[VertexPartition[VD]]) = {
@@ -187,6 +187,7 @@ object ReplicatedVertexView {
   }
 }
 
+private[graphx]
 class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexID], val attrs: Array[VD])
   extends Serializable {
   def iterator: Iterator[(VertexID, VD)] =

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala
index 3bd8b24..f342fd7 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala
@@ -12,6 +12,7 @@ import org.apache.spark.util.collection.PrimitiveVector
  * may be used multiple times in ReplicatedVertexView -- once to ship the vertex attributes and
  * (possibly) once to ship the active-set information.
  */
+private[impl]
 class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) {
 
   val bothAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(true, true)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala
index 1c3c87f..cbd6318 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala
@@ -7,6 +7,7 @@ import org.apache.spark.SparkConf
 import org.apache.spark.graphx._
 import org.apache.spark.serializer._
 
+private[graphx]
 class VertexIDMsgSerializer(conf: SparkConf) extends Serializer {
   override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
 
@@ -27,6 +28,7 @@ class VertexIDMsgSerializer(conf: SparkConf) extends Serializer {
 }
 
 /** A special shuffle serializer for VertexBroadcastMessage[Int]. */
+private[graphx]
 class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer {
   override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
 
@@ -50,6 +52,7 @@ class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer {
 }
 
 /** A special shuffle serializer for VertexBroadcastMessage[Long]. */
+private[graphx]
 class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer {
   override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
 
@@ -73,6 +76,7 @@ class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer {
 }
 
 /** A special shuffle serializer for VertexBroadcastMessage[Double]. */
+private[graphx]
 class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer {
   override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
 
@@ -96,6 +100,7 @@ class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer {
 }
 
 /** A special shuffle serializer for AggregationMessage[Int]. */
+private[graphx]
 class IntAggMsgSerializer(conf: SparkConf) extends Serializer {
   override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
 
@@ -119,6 +124,7 @@ class IntAggMsgSerializer(conf: SparkConf) extends Serializer {
 }
 
 /** A special shuffle serializer for AggregationMessage[Long]. */
+private[graphx]
 class LongAggMsgSerializer(conf: SparkConf) extends Serializer {
   override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
 
@@ -142,6 +148,7 @@ class LongAggMsgSerializer(conf: SparkConf) extends Serializer {
 }
 
 /** A special shuffle serializer for AggregationMessage[Double]. */
+private[graphx]
 class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer {
   override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
 
@@ -168,6 +175,7 @@ class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer {
 // Helper classes to shorten the implementation of those special serializers.
 ////////////////////////////////////////////////////////////////////////////////
 
+private[graphx]
 abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream {
   // The implementation should override this one.
   def writeObject[T](t: T): SerializationStream
@@ -281,6 +289,7 @@ abstract class ShuffleSerializationStream(s: OutputStream) extends Serialization
   override def close(): Unit = s.close()
 }
 
+private[graphx]
 abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream {
   // The implementation should override this one.
   def readObject[T](): T
@@ -371,7 +380,7 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat
   override def close(): Unit = s.close()
 }
 
-sealed trait ShuffleSerializerInstance extends SerializerInstance {
+private[graphx] sealed trait ShuffleSerializerInstance extends SerializerInstance {
 
   override def serialize[T](t: T): ByteBuffer = throw new UnsupportedOperationException
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87f335db/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala
index a6bbf63..cfc3281 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala
@@ -3,5 +3,5 @@ package org.apache.spark.graphx
 import org.apache.spark.util.collection.OpenHashSet
 
 package object impl {
-  type VertexIdToIndexMap = OpenHashSet[VertexID]
+  private[graphx] type VertexIdToIndexMap = OpenHashSet[VertexID]
 }


[23/50] git commit: Improvements in example code for the programming guide as well as adding serialization support for GraphImpl to address issues with failed closure capture.

Posted by pw...@apache.org.
Improvements in example code for the programming guide as well as adding serialization support for GraphImpl to address issues with failed closure capture.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/cfe4a29d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/cfe4a29d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/cfe4a29d

Branch: refs/heads/master
Commit: cfe4a29dcb516ceae5f243ac3b5d0c3a505d7f5a
Parents: ae4b75d
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Mon Jan 13 17:15:21 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Mon Jan 13 17:18:31 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 39 +++++++++++---------
 .../apache/spark/graphx/impl/GraphImpl.scala    |  3 ++
 2 files changed, 25 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cfe4a29d/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index e6afd09..c82c3d7 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -478,24 +478,26 @@ def mapReduceTriplets[A](
 
 The [`mapReduceTriplets`][Graph.mapReduceTriplets] operator takes a user defined map function which
 is applied to each triplet and can yield *messages* destined to either (none or both) vertices in
-the triplet.  We currently only support messages destined to the source or destination vertex of the
-triplet to enable optimized preaggregation.  The user defined `reduce` function combines the
+the triplet.  To facilitate optimized pre-aggregation, we currently only support messages destined
+to the source or destination vertex of the triplet.  The user defined `reduce` function combines the
 messages destined to each vertex.  The `mapReduceTriplets` operator returns a `VertexRDD[A]`
-containing the aggregate message to each vertex.  Vertices that do not receive a message are not
-included in the returned `VertexRDD`.
+containing the aggregate message (of type `A`) destined to each vertex.  Vertices that do not
+receive a message are not included in the returned `VertexRDD`.
 
-> Note that `mapReduceTriplets takes an additional optional `activeSet` (see API docs) which
+> Note that `mapReduceTriplets` takes an additional optional `activeSet` (see API docs) which
 > restricts the map phase to edges adjacent to the vertices in the provided `VertexRDD`. Restricting
 > computation to triplets adjacent to a subset of the vertices is often necessary in incremental
 > iterative computation and is a key part of the GraphX implementation of Pregel.
 
-We can use the `mapReduceTriplets` operator to collect information about adjacent vertices.  For
-example if we wanted to compute the average age of followers who are older that each user we could
-do the following.
+In the following example we use the `mapReduceTriplets` operator to compute the average age of the
+more senior followers of each user.
 
 {% highlight scala %}
-// Graph with age as the vertex property
-val graph: Graph[Double, String] = getFromSomewhereElse()
+// Import Random graph generation library
+import org.apache.spark.graphx.util.GraphGenerators
+// Create a graph with "age" as the vertex property.  Here we use a random graph for simplicity.
+val graph: Graph[Double, Int] =
+  GraphGenerators.logNormalGraph(sc, numVertices = 100).mapVertices( (id, _) => id.toDouble )
 // Compute the number of older followers and their total age
 val olderFollowers: VertexRDD[(Int, Double)] = graph.mapReduceTriplets[(Int, Double)](
   triplet => { // Map Function
@@ -511,13 +513,16 @@ val olderFollowers: VertexRDD[(Int, Double)] = graph.mapReduceTriplets[(Int, Dou
   (a, b) => (a._1 + b._1, a._2 + b._2) // Reduce Function
 )
 // Divide total age by number of older followers to get average age of older followers
-val avgAgeOlderFollowers: VertexRDD[Double] =
-  olderFollowers.mapValues { case (count, totalAge) => totalAge / count }
+val avgAgeOfOlderFollowers: VertexRDD[Double] =
+  olderFollowers.mapValues( (id, value) => value match { case (count, totalAge) => totalAge / count } )
+// Display the results
+avgAgeOfOlderFollowers.collect.foreach(println(_))
 {% endhighlight %}
 
 > Note that the `mapReduceTriplets` operation performs optimally when the messages (and their sums)
 > are constant sized (e.g., floats and addition instead of lists and concatenation).  More
-> precisely, the result of `mapReduceTriplets` should be sub-linear in the degree of each vertex.
+> precisely, the result of `mapReduceTriplets` should ideally be sub-linear in the degree of each
+> vertex.
 
 ### Computing Degree Information
 
@@ -529,13 +534,13 @@ compute the max in, out, and total degrees:
 
 {% highlight scala %}
 // Define a reduce operation to compute the highest degree vertex
-def maxReduce(a: (VertexId, Int), b: (VertexId, Int)): (VertexId, Int) = {
+def max(a: (VertexID, Int), b: (VertexID, Int)): (VertexID, Int) = {
   if (a._2 > b._2) a else b
 }
 // Compute the max degrees
-val maxInDegree: (VertexId, Int)  = graph.inDegrees.reduce(maxReduce)
-val maxOutDegree: (VertexId, Int) = graph.outDegrees.reduce(maxReduce)
-val maxDegrees: (VertexId, Int)   = graph.degrees.reduce(maxReduce)
+val maxInDegree: (VertexID, Int)  = graph.inDegrees.reduce(max)
+val maxOutDegree: (VertexID, Int) = graph.outDegrees.reduce(max)
+val maxDegrees: (VertexID, Int)   = graph.degrees.reduce(max)
 {% endhighlight %}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cfe4a29d/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index c21f893..916eb97 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -32,6 +32,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
     @transient val replicatedVertexView: ReplicatedVertexView[VD])
   extends Graph[VD, ED] with Serializable {
 
+  /** Default construct is provided to support serialization */
+  protected def this() = this(null, null, null, null)
+
   /** Return a RDD that brings edges together with their source and destination vertices. */
   @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = {
     val vdTag = classTag[VD]


[21/50] git commit: Remove aggregateNeighbors

Posted by pw...@apache.org.
Remove aggregateNeighbors


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1bd5cefc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1bd5cefc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1bd5cefc

Branch: refs/heads/master
Commit: 1bd5cefcae2769d48ad5ef4b8197193371c754da
Parents: e2d25d2
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 16:15:10 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 17:03:03 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 17 ------
 .../org/apache/spark/graphx/GraphOps.scala      | 64 ++------------------
 .../org/apache/spark/graphx/GraphOpsSuite.scala | 26 --------
 3 files changed, 5 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1bd5cefc/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 002ba0c..e6afd09 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -519,23 +519,6 @@ val avgAgeOlderFollowers: VertexRDD[Double] =
 > are constant sized (e.g., floats and addition instead of lists and concatenation).  More
 > precisely, the result of `mapReduceTriplets` should be sub-linear in the degree of each vertex.
 
-Because it is often necessary to aggregate information about neighboring vertices we also provide an
-alternative interface defined in [`GraphOps`][GraphOps]:
-
-{% highlight scala %}
-def aggregateNeighbors[A](
-    map: (VertexID, EdgeTriplet[VD, ED]) => Option[A],
-    reduce: (A, A) => A,
-    edgeDir: EdgeDirection)
-  : VertexRDD[A]
-{% endhighlight %}
-
-The `aggregateNeighbors` operator is implemented directly on top of `mapReduceTriplets` but allows
-the user to define the logic in a more vertex centric manner.  Here the `map` function is provided
-the vertex to which the message is sent as well as one of the edges and returns the optional message
-value.  The `edgeDir` determines whether the `map` function is run on `In`, `Out`, or `All` edges
-adjacent to each vertex.
-
 ### Computing Degree Information
 
 A common aggregation task is computing the degree of each vertex: the number of edges adjacent to

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1bd5cefc/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index a0a40e2..578eb33 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -56,60 +56,6 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
   }
 
   /**
-   * Computes a statistic for the neighborhood of each vertex.
-   *
-   * @param mapFunc the function applied to each edge adjacent to each vertex. The mapFunc can
-   * optionally return `None`, in which case it does not contribute to the final sum.
-   * @param reduceFunc the function used to merge the results of each map operation
-   * @param direction the direction of edges to consider (e.g., In, Out, Both).
-   * @tparam A the aggregation type
-   *
-   * @return an RDD containing tuples of vertex identifiers and
-   * their resulting value. Vertices with no neighbors will not appear in the RDD.
-   *
-   * @example We can use this function to compute the average follower
-   * age for each user:
-   *
-   * {{{
-   * val graph: Graph[Int,Int] = GraphLoader.edgeListFile(sc, "webgraph")
-   * val averageFollowerAge: RDD[(Int, Int)] =
-   *   graph.aggregateNeighbors[(Int,Double)](
-   *     (vid, edge) => Some((edge.otherVertex(vid).data, 1)),
-   *     (a, b) => (a._1 + b._1, a._2 + b._2),
-   *     -1,
-   *     EdgeDirection.In)
-   *     .mapValues{ case (sum,followers) => sum.toDouble / followers}
-   * }}}
-   */
-  def aggregateNeighbors[A: ClassTag](
-      mapFunc: (VertexID, EdgeTriplet[VD, ED]) => Option[A],
-      reduceFunc: (A, A) => A,
-      dir: EdgeDirection)
-    : VertexRDD[A] = {
-    // Define a new map function over edge triplets
-    val mf = (et: EdgeTriplet[VD,ED]) => {
-      // Compute the message to the dst vertex
-      val dst =
-        if (dir == EdgeDirection.In || dir == EdgeDirection.Both) {
-          mapFunc(et.dstId, et)
-        } else { Option.empty[A] }
-      // Compute the message to the source vertex
-      val src =
-        if (dir == EdgeDirection.Out || dir == EdgeDirection.Both) {
-          mapFunc(et.srcId, et)
-        } else { Option.empty[A] }
-      // construct the return array
-      (src, dst) match {
-        case (None, None) => Iterator.empty
-        case (Some(srcA),None) => Iterator((et.srcId, srcA))
-        case (None, Some(dstA)) => Iterator((et.dstId, dstA))
-        case (Some(srcA), Some(dstA)) => Iterator((et.srcId, srcA), (et.dstId, dstA))
-      }
-    }
-    graph.mapReduceTriplets(mf, reduceFunc)
-  } // end of aggregateNeighbors
-
-  /**
    * Collect the neighbor vertex ids for each vertex.
    *
    * @param edgeDirection the direction along which to collect
@@ -152,11 +98,11 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    *
    * @return the vertex set of neighboring vertex attributes for each vertex
    */
-  def collectNeighbors(edgeDirection: EdgeDirection) :
-    VertexRDD[ Array[(VertexID, VD)] ] = {
-    val nbrs = graph.aggregateNeighbors[Array[(VertexID,VD)]](
-      (vid, edge) =>
-        Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )),
+  def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexID, VD)]] = {
+    val nbrs = graph.mapReduceTriplets[Array[(VertexID,VD)]](
+      edge => Iterator(
+        (edge.srcId, Array((edge.dstId, edge.dstAttr))),
+        (edge.dstId, Array((edge.srcId, edge.srcAttr)))),
       (a, b) => a ++ b,
       edgeDirection)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1bd5cefc/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
index cd3c0bb..7a90140 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
@@ -8,32 +8,6 @@ import org.scalatest.FunSuite
 
 class GraphOpsSuite extends FunSuite with LocalSparkContext {
 
-  test("aggregateNeighbors") {
-    withSpark { sc =>
-      val n = 3
-      val star =
-        Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID))), 1)
-
-      val indegrees = star.aggregateNeighbors(
-        (vid, edge) => Some(1),
-        (a: Int, b: Int) => a + b,
-        EdgeDirection.In)
-      assert(indegrees.collect().toSet === (1 to n).map(x => (x, 1)).toSet)
-
-      val outdegrees = star.aggregateNeighbors(
-        (vid, edge) => Some(1),
-        (a: Int, b: Int) => a + b,
-        EdgeDirection.Out)
-      assert(outdegrees.collect().toSet === Set((0, n)))
-
-      val noVertexValues = star.aggregateNeighbors[Int](
-        (vid: VertexID, edge: EdgeTriplet[Int, Int]) => None,
-        (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"),
-        EdgeDirection.In)
-      assert(noVertexValues.collect().toSet === Set.empty[(VertexID, Int)])
-    }
-  }
-
   test("joinVertices") {
     withSpark { sc =>
       val vertices =


[12/50] git commit: Add LiveJournalPageRank example

Posted by pw...@apache.org.
Add LiveJournalPageRank example


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/8ca97739
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/8ca97739
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/8ca97739

Branch: refs/heads/master
Commit: 8ca97739741152cce30adfce80aee4462b5a04f2
Parents: 66c9d00
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 12:17:58 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 12:17:58 2014 -0800

----------------------------------------------------------------------
 .../examples/graphx/LiveJournalPageRank.scala   | 49 ++++++++++++++++++++
 1 file changed, 49 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8ca97739/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
new file mode 100644
index 0000000..d58fddf
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.spark.examples.graphx
+
+import org.apache.spark.SparkContext._
+import org.apache.spark._
+import org.apache.spark.graphx._
+import org.apache.spark.graphx.lib.Analytics
+
+/**
+ * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from
+ * http://snap.stanford.edu/data/soc-LiveJournal1.html.
+ */
+object LiveJournalPageRank {
+  def main(args: Array[String]) {
+    if (args.length < 2) {
+      System.err.println(
+        "Usage: LiveJournalPageRank <master> <edge_list_file>\n" +
+          "    [--tol=<tolerance>]\n" +
+          "        The tolerance allowed at convergence (smaller => more accurate). Default is " +
+          "0.001.\n" +
+          "    [--output=<output_file>]\n" +
+          "        If specified, the file to write the ranks to.\n" +
+          "    [--numEPart=<num_edge_partitions>]\n" +
+          "        The number of partitions for the graph's edge RDD. Default is 4.\n" +
+          "    [--partStrategy=RandomVertexCut | EdgePartition1D | EdgePartition2D | " +
+          "CanonicalRandomVertexCut]\n" +
+          "        The way edges are assigned to edge partitions. Default is RandomVertexCut.")
+      System.exit(-1)
+    }
+
+    Analytics.main(args.patch(1, List("pagerank"), 0))
+  }
+}


[10/50] git commit: Use GraphLoader for algorithms examples in doc

Posted by pw...@apache.org.
Use GraphLoader for algorithms examples in doc


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1efe78a1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1efe78a1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1efe78a1

Branch: refs/heads/master
Commit: 1efe78a1013af1aa97d07c18b27f1ccbb90c2790
Parents: d691e9f
Author: Ankur Dave <an...@gmail.com>
Authored: Sun Jan 12 22:03:03 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Sun Jan 12 22:03:03 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 36 ++++++++++++++++++-----------------
 1 file changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1efe78a1/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 572afc1..f2f5a88 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -674,24 +674,22 @@ GraphX includes a set of graph algorithms in to simplify analytics. The algorith
 
 PageRank measures the importance of each vertex in a graph, assuming an edge from *u* to *v* represents an endorsement of *v*'s importance by *u*. For example, if a Twitter user is followed by many others, the user will be ranked highly.
 
-GraphX comes with static and dynamic implementations of PageRank as methods on the [`PageRank` object][PageRank]. Static PageRank runs for a fixed number of iterations, while dynamic PageRank runs until the ranks converge (i.e., stop changing by more than a specified tolerance). GraphX also includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We compute the PageRank of each user as follows:
+GraphX comes with static and dynamic implementations of PageRank as methods on the [`PageRank` object][PageRank]. Static PageRank runs for a fixed number of iterations, while dynamic PageRank runs until the ranks converge (i.e., stop changing by more than a specified tolerance). [`GraphOps`][GraphOps] allows calling these algorithms directly as methods on `Graph`.
+
+GraphX also includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We compute the PageRank of each user as follows:
 
 [PageRank]: api/graphx/index.html#org.apache.spark.graphx.lib.PageRank$
 
 {% highlight scala %}
-// Load the datasets into a graph
+// Load the edges as a graph
+val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt")
+// Run PageRank
+val ranks = graph.pageRank(0.0001).vertices
+// Join the ranks with the usernames
 val users = sc.textFile("graphx/data/users.txt").map { line =>
   val fields = line.split("\\s+")
   (fields(0).toLong, fields(1))
 }
-val followers = sc.textFile("graphx/data/followers.txt").map { line =>
-  val fields = line.split("\\s+")
-  Edge(fields(0).toLong, fields(1).toLong, 1)
-}
-val graph = Graph(users, followers)
-// Run PageRank
-val ranks = graph.pageRank(0.0001).vertices
-// Join the ranks with the usernames
 val ranksByUsername = users.leftOuterJoin(ranks).map {
   case (id, (username, rankOpt)) => (username, rankOpt.getOrElse(0.0))
 }
@@ -707,13 +705,15 @@ The connected components algorithm labels each connected component of the graph
 
 {% highlight scala %}
 // Load the graph as in the PageRank example
-val users = ...
-val followers = ...
-val graph = Graph(users, followers)
+val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt")
 // Find the connected components
 val cc = graph.connectedComponents().vertices
 // Join the connected components with the usernames
-val ccByUsername = graph.vertices.innerJoin(cc) { (id, username, cc) =>
+val users = sc.textFile("graphx/data/users.txt").map { line =>
+  val fields = line.split("\\s+")
+  (fields(0).toLong, fields(1))
+}
+val ccByUsername = users.join(cc).map { case (id, (username, cc)) =>
   (username, cc)
 }
 // Print the result
@@ -728,14 +728,16 @@ A vertex is part of a triangle when it has two adjacent vertices with an edge be
 [Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph@partitionBy(PartitionStrategy):Graph[VD,ED]
 
 {% highlight scala %}
-// Load the graph as in the PageRank example
-val users = ...
 // Load the edges in canonical order and partition the graph for triangle count
 val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(RandomVertexCut)
 // Find the triangle count for each vertex
 val triCounts = graph.triangleCount().vertices
 // Join the triangle counts with the usernames
-val triCountByUsername = graph.vertices.innerJoin(triCounts) { (id, username, tc) =>
+val users = sc.textFile("graphx/data/users.txt").map { line =>
+  val fields = line.split("\\s+")
+  (fields(0).toLong, fields(1))
+}
+val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) =>
   (username, tc)
 }
 // Print the result


[24/50] git commit: Miscel doc update.

Posted by pw...@apache.org.
Miscel doc update.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/02a8f54b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/02a8f54b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/02a8f54b

Branch: refs/heads/master
Commit: 02a8f54bfa4572908d2d605a85e7a5adf9a36fbc
Parents: dc041cd
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 17:40:36 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 17:40:36 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/graphx/Edge.scala    |  2 +-
 .../org/apache/spark/graphx/EdgeTriplet.scala   |  4 +-
 .../scala/org/apache/spark/graphx/Graph.scala   | 75 +++++++---------
 .../org/apache/spark/graphx/GraphLoader.scala   | 14 ++-
 .../apache/spark/graphx/PartitionStrategy.scala |  1 +
 .../scala/org/apache/spark/graphx/Pregel.scala  | 14 +--
 .../spark/graphx/impl/EdgePartition.scala       |  7 +-
 .../apache/spark/graphx/impl/GraphImpl.scala    | 15 ++--
 .../org/apache/spark/graphx/lib/Analytics.scala | 12 +--
 .../spark/graphx/lib/ConnectedComponents.scala  |  2 +-
 .../org/apache/spark/graphx/lib/PageRank.scala  |  2 +-
 .../apache/spark/graphx/lib/SVDPlusPlus.scala   | 95 +++++++++++++-------
 .../lib/StronglyConnectedComponents.scala       |  1 +
 .../apache/spark/graphx/lib/TriangleCount.scala | 43 +++++----
 .../scala/org/apache/spark/graphx/package.scala |  5 +-
 .../spark/graphx/util/BytecodeUtils.scala       |  7 +-
 .../collection/PrimitiveKeyOpenHashMap.scala    |  2 +-
 17 files changed, 158 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
index 85f27d2..6c396c3 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
@@ -13,7 +13,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED]
     /** The vertex id of the target vertex. */
     var dstId: VertexID = 0,
     /** The attribute associated with the edge. */
-    var attr: ED = nullValue[ED])
+    var attr: ED = null.asInstanceOf[ED])
   extends Serializable {
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala
index 057d63a..4253b24 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala
@@ -1,7 +1,5 @@
 package org.apache.spark.graphx
 
-import org.apache.spark.graphx.impl.VertexPartition
-
 /**
  * An edge triplet represents an edge along with the vertex attributes of its neighboring vertices.
  *
@@ -47,5 +45,5 @@ class EdgeTriplet[VD, ED] extends Edge[ED] {
   def vertexAttr(vid: VertexID): VD =
     if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr }
 
-  override def toString() = ((srcId, srcAttr), (dstId, dstAttr), attr).toString()
+  override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString()
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 7d4f0de..d2ba6fd 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -45,7 +45,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
 
   /**
    * An RDD containing the edge triplets, which are edges along with the vertex data associated with
-   * the adjacent vertices.
+   * the adjacent vertices. The caller should use [[edges]] if the vertex data are not needed, i.e.
+   * if only the edge data and adjacent vertex ids are needed.
    *
    * @return an RDD containing edge triplets
    *
@@ -54,13 +55,9 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * different color.
    * {{{
    * type Color = Int
-   * val graph: Graph[Color, Int] = Graph.textFile("hdfs://file.tsv")
+   * val graph: Graph[Color, Int] = GraphLoader.edgeListFile("hdfs://file.tsv")
    * val numInvalid = graph.triplets.map(e => if (e.src.data == e.dst.data) 1 else 0).sum
    * }}}
-   *
-   * @see `edges` if only the edge data and adjacent vertex ids are
-   * required.
-   *
    */
   val triplets: RDD[EdgeTriplet[VD, ED]]
 
@@ -68,9 +65,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * Caches the vertices and edges associated with this graph at the specified storage level.
    *
    * @param newLevel the level at which to cache the graph.
-
-   * @return A reference to this graph for convenience.
    *
+   * @return A reference to this graph for convenience.
    */
   def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED]
 
@@ -159,8 +155,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * @tparam ED2 the new edge data type
    *
    */
-  def mapEdges[ED2: ClassTag](
-      map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2]
+  def mapEdges[ED2: ClassTag](map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2])
+    : Graph[VD, ED2]
 
   /**
    * Transforms each edge attribute using the map function, passing it the adjacent vertex attributes
@@ -203,9 +199,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * @tparam ED2 the new edge data type
    *
    */
-  def mapTriplets[ED2: ClassTag](
-      map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]):
-    Graph[VD, ED2]
+  def mapTriplets[ED2: ClassTag](map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2])
+    : Graph[VD, ED2]
 
   /**
    * Reverses all edges in the graph.  If this graph contains an edge from a to b then the returned
@@ -233,8 +228,10 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * @return the subgraph containing only the vertices and edges that
    * satisfy the predicates
    */
-  def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true),
-    vpred: (VertexID, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED]
+  def subgraph(
+      epred: EdgeTriplet[VD,ED] => Boolean = (x => true),
+      vpred: (VertexID, VD) => Boolean = ((v, d) => true))
+    : Graph[VD, ED]
 
   /**
    * Restricts the graph to only the vertices and edges that are also in `other`, but keeps the
@@ -249,14 +246,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * Merges multiple edges between two vertices into a single edge. For correct results, the graph
    * must have been partitioned using [[partitionBy]].
    *
-   * @tparam ED2 the type of the resulting edge data after grouping.
-   *
-   * @param f the user-supplied commutative associative function to merge edge attributes for
-   * duplicate edges.
+   * @param merge the user-supplied commutative associative function to merge edge attributes
+   *              for duplicate edges.
    *
    * @return The resulting graph with a single edge for each (source, dest) vertex pair.
    */
-  def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED]
+  def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED]
 
   /**
    * Computes statistics about the neighboring edges and vertices of each vertex.  The user supplied
@@ -270,7 +265,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * more messages to neighboring vertices
    *
    * @param reduceFunc the user defined reduce function which should
-   * be commutative and assosciative and is used to combine the output
+   * be commutative and associative and is used to combine the output
    * of the map phase
    *
    * @param activeSetOpt optionally, a set of "active" vertices and a direction of edges to consider
@@ -301,21 +296,20 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
 
   /**
    * Joins the vertices with entries in the `table` RDD and merges the results using `mapFunc`.  The
-   * input table should contain at most one entry for each vertex.  If no entry in `table` is
+   * input table should contain at most one entry for each vertex.  If no entry in `other` is
    * provided for a particular vertex in the graph, the map function receives `None`.
    *
    * @tparam U the type of entry in the table of updates
    * @tparam VD2 the new vertex value type
    *
-   * @param table the table to join with the vertices in the graph.
-   * The table should contain at most one entry for each vertex.
-   *
-   * @param mapFunc the function used to compute the new vertex
-   * values.  The map function is invoked for all vertices, even those
-   * that do not have a corresponding entry in the table.
+   * @param other the table to join with the vertices in the graph.
+   *              The table should contain at most one entry for each vertex.
+   * @param mapFunc the function used to compute the new vertex values.
+   *                The map function is invoked for all vertices, even those
+   *                that do not have a corresponding entry in the table.
    *
    * @example This function is used to update the vertices with new values based on external data.
-   * For example we could add the out-degree to each vertex record:
+   *          For example we could add the out-degree to each vertex record:
    *
    * {{{
    * val rawGraph: Graph[_, _] = Graph.textFile("webgraph")
@@ -324,20 +318,20 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    *   (vid, data, optDeg) => optDeg.getOrElse(0)
    * }
    * }}}
-   *
    */
-  def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(VertexID, U)])
+  def outerJoinVertices[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)])
       (mapFunc: (VertexID, VD, Option[U]) => VD2)
     : Graph[VD2, ED]
 
+  /**
+   * The associated [[GraphOps]] object.
+   */
   // Save a copy of the GraphOps object so there is always one unique GraphOps object
   // for a given Graph object, and thus the lazy vals in GraphOps would work as intended.
   val ops = new GraphOps(this)
 } // end of Graph
 
 
-
-
 /**
  * The Graph object contains a collection of routines used to construct graphs from RDDs.
  */
@@ -357,7 +351,8 @@ object Graph {
   def fromEdgeTuples[VD: ClassTag](
       rawEdges: RDD[(VertexID, VertexID)],
       defaultValue: VD,
-      uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = {
+      uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] =
+  {
     val edges = rawEdges.map(p => Edge(p._1, p._2, 1))
     val graph = GraphImpl(edges, defaultValue)
     uniqueEdges match {
@@ -391,10 +386,8 @@ object Graph {
    * @tparam ED the edge attribute type
    * @param vertices the "set" of vertices and their attributes
    * @param edges the collection of edges in the graph
-   * @param defaultVertexAttr the default vertex attribute to use for
-   * vertices that are mentioned in edges but not in vertices
-   * @param partitionStrategy the partition strategy to use when
-   * partitioning the edges
+   * @param defaultVertexAttr the default vertex attribute to use for vertices that are
+   *                          mentioned in edges but not in vertices
    */
   def apply[VD: ClassTag, ED: ClassTag](
       vertices: RDD[(VertexID, VD)],
@@ -406,9 +399,9 @@ object Graph {
   /**
    * Implicitly extracts the [[GraphOps]] member from a graph.
    *
-   * To improve modularity the Graph type only contains a small set of basic operations.  All the
-   * convenience operations are defined in the [[GraphOps]] class which may be shared across multiple
-   * graph implementations.
+   * To improve modularity the Graph type only contains a small set of basic operations.
+   * All the convenience operations are defined in the [[GraphOps]] class which may be
+   * shared across multiple graph implementations.
    */
   implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops
 } // end of Graph object

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
index 3c06a40..7bdb101 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
@@ -1,12 +1,7 @@
 package org.apache.spark.graphx
 
-import java.util.{Arrays => JArrays}
-import scala.reflect.ClassTag
-
-import org.apache.spark.graphx.impl.EdgePartitionBuilder
 import org.apache.spark.{Logging, SparkContext}
-import org.apache.spark.graphx.impl.{EdgePartition, GraphImpl}
-import org.apache.spark.util.collection.PrimitiveVector
+import org.apache.spark.graphx.impl.{EdgePartitionBuilder, GraphImpl}
 
 /**
  * Provides utilities for loading [[Graph]]s from files.
@@ -31,19 +26,20 @@ object GraphLoader extends Logging {
    * 1    8
    * }}}
    *
-   * @param sc
+   * @param sc SparkContext
    * @param path the path to the file (e.g., /home/data/file or hdfs://file)
    * @param canonicalOrientation whether to orient edges in the positive
    *        direction
    * @param minEdgePartitions the number of partitions for the
    *        the edge RDD
-   * @tparam ED
    */
   def edgeListFile(
       sc: SparkContext,
       path: String,
       canonicalOrientation: Boolean = false,
-      minEdgePartitions: Int = 1): Graph[Int, Int] = {
+      minEdgePartitions: Int = 1)
+    : Graph[Int, Int] =
+  {
     val startTime = System.currentTimeMillis
 
     // Parse the edge data table directly into edge partitions

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
index fc7635a..b9ccd87 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
@@ -5,6 +5,7 @@ package org.apache.spark.graphx
  * vertex IDs.
  */
 sealed trait PartitionStrategy extends Serializable {
+  /** Returns the partition number for a given edge. */
   def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
index 83e28d0..ce4eb53 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
@@ -89,14 +89,16 @@ object Pregel {
    *
    */
   def apply[VD: ClassTag, ED: ClassTag, A: ClassTag]
-    (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue,
-      activeDirection: EdgeDirection = EdgeDirection.Out)(
-      vprog: (VertexID, VD, A) => VD,
+     (graph: Graph[VD, ED],
+      initialMsg: A,
+      maxIterations: Int = Int.MaxValue,
+      activeDirection: EdgeDirection = EdgeDirection.Out)
+     (vprog: (VertexID, VD, A) => VD,
       sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
       mergeMsg: (A, A) => A)
-    : Graph[VD, ED] = {
-
-    var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache()
+    : Graph[VD, ED] =
+  {
+    var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)).cache()
     // compute the messages
     var messages = g.mapReduceTriplets(sendMsg, mergeMsg)
     var activeMessages = messages.count()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
index a03e73e..d4d7162 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
@@ -100,10 +100,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
    */
   def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = {
     val builder = new EdgePartitionBuilder[ED]
-    var firstIter: Boolean = true
-    var currSrcId: VertexID = nullValue[VertexID]
-    var currDstId: VertexID = nullValue[VertexID]
-    var currAttr: ED = nullValue[ED]
+    var currSrcId: VertexID = null.asInstanceOf[VertexID]
+    var currDstId: VertexID = null.asInstanceOf[VertexID]
+    var currAttr: ED = null.asInstanceOf[ED]
     var i = 0
     while (i < size) {
       if (i > 0 && currSrcId == srcIds(i) && currDstId == dstIds(i)) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 6a2abc7..9e39519 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -249,8 +249,8 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
 
     // For each vertex, replicate its attribute only to partitions where it is
     // in the relevant position in an edge.
-    val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr")
-    val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr")
+    val mapUsesSrcAttr = accessesVertexAttr(mapFunc, "srcAttr")
+    val mapUsesDstAttr = accessesVertexAttr(mapFunc, "dstAttr")
     val vs = activeSetOpt match {
       case Some((activeSet, _)) =>
         replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet)
@@ -308,10 +308,12 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
   } // end of mapReduceTriplets
 
   override def outerJoinVertices[U: ClassTag, VD2: ClassTag]
-      (updates: RDD[(VertexID, U)])(updateF: (VertexID, VD, Option[U]) => VD2): Graph[VD2, ED] = {
+      (other: RDD[(VertexID, U)])
+      (updateF: (VertexID, VD, Option[U]) => VD2): Graph[VD2, ED] =
+  {
     if (classTag[VD] equals classTag[VD2]) {
       // updateF preserves type, so we can use incremental replication
-      val newVerts = vertices.leftJoin(updates)(updateF)
+      val newVerts = vertices.leftJoin(other)(updateF)
       val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts)
       val newReplicatedVertexView = new ReplicatedVertexView[VD2](
         changedVerts, edges, routingTable,
@@ -319,12 +321,13 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
       new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView)
     } else {
       // updateF does not preserve type, so we must re-replicate all vertices
-      val newVerts = vertices.leftJoin(updates)(updateF)
+      val newVerts = vertices.leftJoin(other)(updateF)
       GraphImpl(newVerts, edges, routingTable)
     }
   }
 
-  private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = {
+  /** Test whether the closure accesses the the attribute with name `attrName`. */
+  private def accessesVertexAttr(closure: AnyRef, attrName: String): Boolean = {
     try {
       BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName)
     } catch {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
index 8c35f42..d5e1de1 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
@@ -37,14 +37,12 @@ object Analytics extends Logging {
       case "pagerank" =>
         var tol: Float = 0.001F
         var outFname = ""
-        var numVPart = 4
         var numEPart = 4
         var partitionStrategy: Option[PartitionStrategy] = None
 
         options.foreach{
           case ("tol", v) => tol = v.toFloat
           case ("output", v) => outFname = v
-          case ("numVPart", v) => numVPart = v.toInt
           case ("numEPart", v) => numEPart = v.toInt
           case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v))
           case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt)
@@ -90,16 +88,12 @@ object Analytics extends Logging {
           case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt)
         }
 
-        if(!isDynamic && numIter == Int.MaxValue) {
+        if (!isDynamic && numIter == Int.MaxValue) {
           println("Set number of iterations!")
           sys.exit(1)
         }
         println("======================================")
         println("|      Connected Components          |")
-        println("--------------------------------------")
-        println(" Using parameters:")
-        println(" \tDynamic:  " + isDynamic)
-        println(" \tNumIter:  " + numIter)
         println("======================================")
 
         val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")", conf)
@@ -112,20 +106,18 @@ object Analytics extends Logging {
         sc.stop()
 
       case "triangles" =>
-        var numVPart = 4
         var numEPart = 4
         // TriangleCount requires the graph to be partitioned
         var partitionStrategy: PartitionStrategy = RandomVertexCut
 
         options.foreach{
           case ("numEPart", v) => numEPart = v.toInt
-          case ("numVPart", v) => numVPart = v.toInt
           case ("partStrategy", v) => partitionStrategy = pickPartitioner(v)
           case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt)
         }
         println("======================================")
         println("|      Triangle Count                |")
-        println("--------------------------------------")
+        println("======================================")
         val sc = new SparkContext(host, "TriangleCount(" + fname + ")", conf)
         val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true,
           minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
index d078d2a..da03d99 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
@@ -4,7 +4,7 @@ import scala.reflect.ClassTag
 
 import org.apache.spark.graphx._
 
-
+/** Connected components algorithm. */
 object ConnectedComponents {
   /**
    * Compute the connected component membership of each vertex and return a graph with the vertex

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
index cf95267..853ef38 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
@@ -5,7 +5,7 @@ import scala.reflect.ClassTag
 import org.apache.spark.Logging
 import org.apache.spark.graphx._
 
-
+/** PageRank algorithm implementation. */
 object PageRank extends Logging {
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
index f5570da..fa6b1db 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
@@ -1,11 +1,12 @@
 package org.apache.spark.graphx.lib
 
-import org.apache.spark.rdd._
-import org.apache.spark.graphx._
 import scala.util.Random
 import org.apache.commons.math.linear._
+import org.apache.spark.rdd._
+import org.apache.spark.graphx._
 
-class SVDPlusPlusConf( // SVDPlusPlus parameters
+/** Configuration parameters for SVDPlusPlus. */
+class SVDPlusPlusConf(
   var rank: Int,
   var maxIters: Int,
   var minVal: Double,
@@ -15,11 +16,15 @@ class SVDPlusPlusConf( // SVDPlusPlus parameters
   var gamma6: Double,
   var gamma7: Double) extends Serializable
 
+/** Implementation of SVD++ algorithm. */
 object SVDPlusPlus {
   /**
-   * Implement SVD++ based on "Factorization Meets the Neighborhood: a Multifaceted Collaborative Filtering Model",
-   * paper is available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]].
-   * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), see the details on page 6.
+   * Implement SVD++ based on "Factorization Meets the Neighborhood:
+   * a Multifaceted Collaborative Filtering Model",
+   * available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]].
+   *
+   * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)),
+   * see the details on page 6.
    *
    * @param edges edges for constructing the graph
    *
@@ -27,16 +32,16 @@ object SVDPlusPlus {
    *
    * @return a graph with vertex attributes containing the trained model
    */
-
-  def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = {
-
-    // generate default vertex attribute
+  def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf)
+    : (Graph[(RealVector, RealVector, Double, Double), Double], Double) =
+  {
+    // Generate default vertex attribute
     def defaultF(rank: Int): (RealVector, RealVector, Double, Double) = {
       val v1 = new ArrayRealVector(rank)
       val v2 = new ArrayRealVector(rank)
       for (i <- 0 until rank) {
-        v1.setEntry(i, Random.nextDouble)
-        v2.setEntry(i, Random.nextDouble)
+        v1.setEntry(i, Random.nextDouble())
+        v2.setEntry(i, Random.nextDouble())
       }
       (v1, v2, 0.0, 0.0)
     }
@@ -49,14 +54,18 @@ object SVDPlusPlus {
     // construct graph
     var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache()
 
-    // calculate initial bias and norm
-    var t0 = g.mapReduceTriplets(et =>
-      Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))), (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2))
-    g = g.outerJoinVertices(t0) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) =>
-      (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1))
+    // Calculate initial bias and norm
+    val t0 = g.mapReduceTriplets(
+      et => Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))),
+      (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2))
+
+    g = g.outerJoinVertices(t0) {
+      (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) =>
+        (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1))
     }
 
-    def mapTrainF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double])
+    def mapTrainF(conf: SVDPlusPlusConf, u: Double)
+        (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double])
       : Iterator[(VertexID, (RealVector, RealVector, Double))] = {
       val (usr, itm) = (et.srcAttr, et.dstAttr)
       val (p, q) = (usr._1, itm._1)
@@ -64,31 +73,49 @@ object SVDPlusPlus {
       pred = math.max(pred, conf.minVal)
       pred = math.min(pred, conf.maxVal)
       val err = et.attr - pred
-      val updateP = ((q.mapMultiply(err)).subtract(p.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2)
-      val updateQ = ((usr._2.mapMultiply(err)).subtract(q.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2)
-      val updateY = ((q.mapMultiply(err * usr._4)).subtract((itm._2).mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2)
+      val updateP = q.mapMultiply(err)
+        .subtract(p.mapMultiply(conf.gamma7))
+        .mapMultiply(conf.gamma2)
+      val updateQ = usr._2.mapMultiply(err)
+        .subtract(q.mapMultiply(conf.gamma7))
+        .mapMultiply(conf.gamma2)
+      val updateY = q.mapMultiply(err * usr._4)
+        .subtract(itm._2.mapMultiply(conf.gamma7))
+        .mapMultiply(conf.gamma2)
       Iterator((et.srcId, (updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)),
         (et.dstId, (updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1)))
     }
 
     for (i <- 0 until conf.maxIters) {
-      // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes
+      // Phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes
       g.cache()
-      var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2))
-      g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) =>
-        if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd
+      val t1 = g.mapReduceTriplets(
+        et => Iterator((et.srcId, et.dstAttr._2)),
+        (g1: RealVector, g2: RealVector) => g1.add(g2))
+      g = g.outerJoinVertices(t1) {
+        (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) =>
+          if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd
       }
-      // phase 2, update p for user nodes and q, y for item nodes
+
+      // Phase 2, update p for user nodes and q, y for item nodes
       g.cache()
-      val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) =>
-        (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3))
-      g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) =>
-        (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4)
+      val t2 = g.mapReduceTriplets(
+        mapTrainF(conf, u),
+        (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) =>
+          (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3))
+      g = g.outerJoinVertices(t2) {
+        (vid: VertexID,
+         vd: (RealVector, RealVector, Double, Double),
+         msg: Option[(RealVector, RealVector, Double)]) =>
+          (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4)
       }
     }
 
     // calculate error on training set
-    def mapTestF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = {
+    def mapTestF(conf: SVDPlusPlusConf, u: Double)
+        (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double])
+      : Iterator[(VertexID, Double)] =
+    {
       val (usr, itm) = (et.srcAttr, et.dstAttr)
       val (p, q) = (usr._1, itm._1)
       var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2)
@@ -99,9 +126,11 @@ object SVDPlusPlus {
     }
     g.cache()
     val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2)
-    g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) =>
-      if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd
+    g = g.outerJoinVertices(t3) {
+      (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) =>
+        if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd
     }
+
     (g, u)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
index 43c4b9c..1184750 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
@@ -4,6 +4,7 @@ import scala.reflect.ClassTag
 
 import org.apache.spark.graphx._
 
+/** Strongly connected components algorithm implementation. */
 object StronglyConnectedComponents {
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
index 58da9e3..f87eab9 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
@@ -4,27 +4,26 @@ import scala.reflect.ClassTag
 
 import org.apache.spark.graphx._
 
-
+/**
+ * Compute the number of triangles passing through each vertex.
+ *
+ * The algorithm is relatively straightforward and can be computed in three steps:
+ *
+ * 1) Compute the set of neighbors for each vertex
+ * 2) For each edge compute the intersection of the sets and send the
+ *    count to both vertices.
+ * 3) Compute the sum at each vertex and divide by two since each
+ *    triangle is counted twice.
+ *
+ * Note that the input graph should have its edges in canonical direction
+ * (i.e. the `sourceId` less than `destId`). Also the graph must have been partitioned
+ * using [[org.apache.spark.graphx.Graph#partitionBy]].
+ */
 object TriangleCount {
-  /**
-   * Compute the number of triangles passing through each vertex.
-   *
-   * The algorithm is relatively straightforward and can be computed in three steps:
-   *
-   * 1) Compute the set of neighbors for each vertex
-   * 2) For each edge compute the intersection of the sets and send the
-   *    count to both vertices.
-   * 3) Compute the sum at each vertex and divide by two since each
-   *    triangle is counted twice.
-   *
-   *
-   * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned
-   * using [[org.apache.spark.graphx.Graph#partitionBy]], and its edges must be in canonical
-   * orientation (srcId < dstId).
-   */
+
   def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = {
     // Remove redundant edges
-    val g = graph.groupEdges((a, b) => a).cache
+    val g = graph.groupEdges((a, b) => a).cache()
 
     // Construct set representations of the neighborhoods
     val nbrSets: VertexRDD[VertexSet] =
@@ -56,8 +55,10 @@ object TriangleCount {
       val iter = smallSet.iterator
       var counter: Int = 0
       while (iter.hasNext) {
-        val vid = iter.next
-        if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) { counter += 1 }
+        val vid = iter.next()
+        if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) {
+          counter += 1
+        }
       }
       Iterator((et.srcId, counter), (et.dstId, counter))
     }
@@ -71,7 +72,5 @@ object TriangleCount {
         assert((dblCount & 1) == 0)
         dblCount / 2
     }
-
   } // end of TriangleCount
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/package.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala
index e70d2fd..60dfc1d 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala
@@ -2,6 +2,7 @@ package org.apache.spark
 
 import org.apache.spark.util.collection.OpenHashSet
 
+/** GraphX is a graph processing framework built on top of Spark. */
 package object graphx {
   /**
    * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need
@@ -9,11 +10,9 @@ package object graphx {
    */
   type VertexID = Long
 
+  /** Integer identifer of a graph partition. */
   // TODO: Consider using Char.
   type PartitionID = Int
 
   private[graphx] type VertexSet = OpenHashSet[VertexID]
-
-  /** Returns the default null-like value for a data type T. */
-  private[graphx] def nullValue[T] = null.asInstanceOf[T]
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
index ec8d534..1c5b234 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
@@ -10,8 +10,11 @@ import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor}
 import org.objectweb.asm.Opcodes._
 
 
-
-private[spark] object BytecodeUtils {
+/**
+ * Includes an utility function to test whether a function accesses a specific attribute
+ * of an object.
+ */
+private[graphx] object BytecodeUtils {
 
   /**
    * Test whether the given closure invokes the specified method in the specified class.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/02a8f54b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala
index 1088944..7b02e2e 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala
@@ -28,7 +28,7 @@ import scala.reflect._
  *
  * Under the hood, it uses our OpenHashSet implementation.
  */
-private[spark]
+private[graphx]
 class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag,
                               @specialized(Long, Int, Double) V: ClassTag](
     val keySet: OpenHashSet[K], var _values: Array[V])


[41/50] git commit: Remove K-Core and LDA sections from guide; they are unimplemented

Posted by pw...@apache.org.
Remove K-Core and LDA sections from guide; they are unimplemented


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/e14a14bc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/e14a14bc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/e14a14bc

Branch: refs/heads/master
Commit: e14a14bcde1637af04cc4c3bd708fed5670e4959
Parents: c28e5a0
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 21:12:58 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:12:58 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e14a14bc/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 91cc5b6..69cadc1 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -848,10 +848,6 @@ val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) =
 println(triCountByUsername.collect().mkString("\n"))
 {% endhighlight %}
 
-## K-Core
-
-## LDA
-
 <p style="text-align: center;">
   <img src="img/tables_and_graphs.png"
        title="Tables and Graphs"


[38/50] git commit: Fix infinite loop in GraphGenerators.generateRandomEdges

Posted by pw...@apache.org.
Fix infinite loop in GraphGenerators.generateRandomEdges

The loop occurred when numEdges < numVertices. This commit fixes it by
allowing generateRandomEdges to generate a multigraph.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c6023bee
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c6023bee
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c6023bee

Branch: refs/heads/master
Commit: c6023bee60cee06b3dd31bb8253da6e07862c13d
Parents: 84d6af8
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 21:00:25 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:02:37 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/graphx/util/GraphGenerators.scala      | 9 +--------
 1 file changed, 1 insertion(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c6023bee/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
index dbea233..57422ce 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
@@ -44,14 +44,7 @@ object GraphGenerators {
 
   def generateRandomEdges(src: Int, numEdges: Int, maxVertexID: Int): Array[Edge[Int]] = {
     val rand = new Random()
-    var dsts: Set[Int] = Set()
-    while (dsts.size < numEdges) {
-      val nextDst = rand.nextInt(maxVertexID)
-      if (nextDst != src) {
-        dsts += nextDst
-      }
-    }
-    dsts.map(dst => Edge[Int](src, dst, 1)).toArray
+    Array.fill(maxVertexID) { Edge[Int](src, rand.nextInt(maxVertexID), 1) }
   }
 
   /**


[48/50] git commit: adding documentation about EdgeRDD

Posted by pw...@apache.org.
adding documentation about EdgeRDD


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/4bafc4f4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/4bafc4f4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/4bafc4f4

Branch: refs/heads/master
Commit: 4bafc4f41f5c5ed686c024d5f49cf31bbc08ce88
Parents: af645be
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Mon Jan 13 22:55:26 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Mon Jan 13 22:55:54 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 42 +++++++++++++++++++++++++++++++++--
 1 file changed, 40 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4bafc4f4/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index a7ab003..9fbde4e 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -811,10 +811,34 @@ setB.count
 val setC: VertexRDD[Double] = setA.innerJoin(setB)((id, a, b) => a + b)
 {% endhighlight %}
 
+## EdgeRDDs
+
+The `EdgeRDD[ED]`, which extends `RDD[Edge[ED]]` is considerably simpler than the `VertexRDD`.
+GraphX organizes the edges in blocks partitioned using one of the various partitioning strategies
+defined in [`PartitionStrategy`][PartitionStrategy].  Within each partition, edge attributes and
+adjacency structure, are stored separately enabling maximum reuse when changing attribute values.
+
+[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy
+
+The three additional functions exposed by the `EdgeRDD` are:
+{% highlight scala %}
+// Transform the edge attributes while preserving the structure
+def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2]
+// Revere the edges reusing both attributes and structure
+def reverse: EdgeRDD[ED]
+// Join two `EdgeRDD`s partitioned using the same partitioning strategy.
+def innerJoin[ED2, ED3](other: EdgeRDD[ED2])(f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3]
+{% endhighlight %}
+
+In most applications we have found that operations on the `EdgeRDD` are accomplished through the
+graph or rely on operations defined in the base `RDD` class.
+
 # Optimized Representation
 
-This section should give some intuition about how GraphX works and how that affects the user (e.g.,
-things to worry about.)
+While a detailed description of the optimizations used in the GraphX representation of distributed
+graphs is beyond the scope of this guide, some high-level understanding may aid in the design of
+scalable algorithms as well as optimal use of the API.  GraphX adopts a vertex-cut approach to
+distributed graph partitioning:
 
 <p style="text-align: center;">
   <img src="img/edge_cut_vs_vertex_cut.png"
@@ -824,6 +848,15 @@ things to worry about.)
   <!-- Images are downsized intentionally to improve quality on retina displays -->
 </p>
 
+Rather than splitting graphs along edges, GraphX partitions the graph along vertices which can
+reduce both the communication and storage overhead.  Logically, this corresponds to assigning edges
+to machines and allowing vertices to span multiple machines.  The exact method of assigning edges
+depends on the [`PartitionStrategy`][PartitionStrategy] and there are several tradeoffs to the
+various heuristics.  Users can choose between different strategies by repartitioning the graph with
+the [`Graph.partitionBy`][Graph.partitionBy] operator.
+
+[Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED]
+
 <p style="text-align: center;">
   <img src="img/vertex_routing_edge_tables.png"
        title="RDD Graph Representation"
@@ -832,6 +865,11 @@ things to worry about.)
   <!-- Images are downsized intentionally to improve quality on retina displays -->
 </p>
 
+Once the edges have be partitioned the key challenge to efficient graph-parallel computation is
+efficiently joining vertex attributes with the edges.  Because real-world graphs typically have more
+edges than vertices, we move vertex attributes to the edges.
+
+
 
 
 


[50/50] git commit: Merge pull request #367 from ankurdave/graphx

Posted by pw...@apache.org.
Merge pull request #367 from ankurdave/graphx

GraphX: Unifying Graphs and Tables

GraphX extends Spark's distributed fault-tolerant collections API and interactive console with a new graph API which leverages recent advances in graph systems (e.g., [GraphLab](http://graphlab.org)) to enable users to easily and interactively build, transform, and reason about graph structured data at scale. See http://amplab.github.io/graphx/.

Thanks to @jegonzal, @rxin, @ankurdave, @dcrankshaw, @jianpingjwang, @amatsukawa, @kellrott, and @adamnovak.

Tasks left:
- [x] Graph-level uncache
- [x] Uncache previous iterations in Pregel
- [x] ~~Uncache previous iterations in GraphLab~~ (postponed to post-release)
- [x] - Describe GC issue with GraphLab
- [ ] Write `docs/graphx-programming-guide.md`
- [x] - Mention future Bagel support in docs
- [ ] - Section on caching/uncaching in docs: As with Spark, cache something that is used more than once. In an iterative algorithm, try to cache and force (i.e., materialize) something every iteration, then uncache the cached things that depended on the newly materialized RDD but that won't be referenced again.
- [x] Undo modifications to core collections and instead copy them to org.apache.spark.graphx
- [x] Make Graph serializable to work around capture in Spark shell
- [x] Rename graph -> graphx in package name and subproject
- [x] Remove standalone PageRank
- [x] ~~Fix amplab/graphx#52 by checking `iter.hasNext`~~


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/4a805aff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/4a805aff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/4a805aff

Branch: refs/heads/master
Commit: 4a805aff5e381752afb2bfd579af908d623743ed
Parents: 945fe7a 80e73ed
Author: Patrick Wendell <pw...@gmail.com>
Authored: Mon Jan 13 22:58:38 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Mon Jan 13 22:58:38 2014 -0800

----------------------------------------------------------------------
 bin/compute-classpath.sh                        |    2 +
 .../org/apache/spark/rdd/PairRDDFunctions.scala |    2 +-
 .../main/scala/org/apache/spark/rdd/RDD.scala   |    5 +
 .../apache/spark/util/collection/BitSet.scala   |   87 +-
 .../spark/util/collection/OpenHashSet.scala     |   23 +-
 docs/_layouts/global.html                       |    8 +-
 docs/_plugins/copy_api_dirs.rb                  |    2 +-
 docs/api.md                                     |    1 +
 docs/bagel-programming-guide.md                 |   10 +-
 docs/graphx-programming-guide.md                | 1003 ++++++++++++++++++
 docs/img/data_parallel_vs_graph_parallel.png    |  Bin 0 -> 432725 bytes
 docs/img/edge-cut.png                           |  Bin 0 -> 12563 bytes
 docs/img/edge_cut_vs_vertex_cut.png             |  Bin 0 -> 79745 bytes
 docs/img/graph_analytics_pipeline.png           |  Bin 0 -> 427220 bytes
 docs/img/graph_parallel.png                     |  Bin 0 -> 92288 bytes
 docs/img/graphx_figures.pptx                    |  Bin 0 -> 1123363 bytes
 docs/img/graphx_logo.png                        |  Bin 0 -> 40324 bytes
 docs/img/graphx_performance_comparison.png      |  Bin 0 -> 166343 bytes
 docs/img/property_graph.png                     |  Bin 0 -> 225151 bytes
 docs/img/tables_and_graphs.png                  |  Bin 0 -> 166265 bytes
 docs/img/triplet.png                            |  Bin 0 -> 31489 bytes
 docs/img/vertex-cut.png                         |  Bin 0 -> 12246 bytes
 docs/img/vertex_routing_edge_tables.png         |  Bin 0 -> 570007 bytes
 docs/index.md                                   |    4 +-
 .../examples/graphx/LiveJournalPageRank.scala   |   49 +
 graphx/data/followers.txt                       |    8 +
 graphx/data/users.txt                           |    7 +
 graphx/pom.xml                                  |   67 ++
 .../scala/org/apache/spark/graphx/Edge.scala    |   45 +
 .../org/apache/spark/graphx/EdgeDirection.scala |   44 +
 .../scala/org/apache/spark/graphx/EdgeRDD.scala |  102 ++
 .../org/apache/spark/graphx/EdgeTriplet.scala   |   49 +
 .../scala/org/apache/spark/graphx/Graph.scala   |  405 +++++++
 .../spark/graphx/GraphKryoRegistrator.scala     |   31 +
 .../org/apache/spark/graphx/GraphLoader.scala   |   72 ++
 .../org/apache/spark/graphx/GraphOps.scala      |  301 ++++++
 .../apache/spark/graphx/PartitionStrategy.scala |  103 ++
 .../scala/org/apache/spark/graphx/Pregel.scala  |  139 +++
 .../org/apache/spark/graphx/VertexRDD.scala     |  347 ++++++
 .../spark/graphx/impl/EdgePartition.scala       |  220 ++++
 .../graphx/impl/EdgePartitionBuilder.scala      |   45 +
 .../spark/graphx/impl/EdgeTripletIterator.scala |   42 +
 .../apache/spark/graphx/impl/GraphImpl.scala    |  379 +++++++
 .../spark/graphx/impl/MessageToPartition.scala  |   98 ++
 .../graphx/impl/ReplicatedVertexView.scala      |  195 ++++
 .../apache/spark/graphx/impl/RoutingTable.scala |   65 ++
 .../apache/spark/graphx/impl/Serializers.scala  |  395 +++++++
 .../spark/graphx/impl/VertexPartition.scala     |  261 +++++
 .../org/apache/spark/graphx/impl/package.scala  |    7 +
 .../org/apache/spark/graphx/lib/Analytics.scala |  136 +++
 .../spark/graphx/lib/ConnectedComponents.scala  |   38 +
 .../org/apache/spark/graphx/lib/PageRank.scala  |  147 +++
 .../apache/spark/graphx/lib/SVDPlusPlus.scala   |  138 +++
 .../lib/StronglyConnectedComponents.scala       |   94 ++
 .../apache/spark/graphx/lib/TriangleCount.scala |   76 ++
 .../scala/org/apache/spark/graphx/package.scala |   18 +
 .../spark/graphx/util/BytecodeUtils.scala       |  117 ++
 .../spark/graphx/util/GraphGenerators.scala     |  218 ++++
 .../collection/PrimitiveKeyOpenHashMap.scala    |  153 +++
 graphx/src/test/resources/log4j.properties      |   28 +
 .../org/apache/spark/graphx/GraphOpsSuite.scala |   66 ++
 .../org/apache/spark/graphx/GraphSuite.scala    |  273 +++++
 .../apache/spark/graphx/LocalSparkContext.scala |   28 +
 .../org/apache/spark/graphx/PregelSuite.scala   |   41 +
 .../apache/spark/graphx/SerializerSuite.scala   |  183 ++++
 .../apache/spark/graphx/VertexRDDSuite.scala    |   85 ++
 .../spark/graphx/impl/EdgePartitionSuite.scala  |   76 ++
 .../graphx/impl/VertexPartitionSuite.scala      |  113 ++
 .../graphx/lib/ConnectedComponentsSuite.scala   |  113 ++
 .../apache/spark/graphx/lib/PageRankSuite.scala |  119 +++
 .../spark/graphx/lib/SVDPlusPlusSuite.scala     |   31 +
 .../lib/StronglyConnectedComponentsSuite.scala  |   57 +
 .../spark/graphx/lib/TriangleCountSuite.scala   |   70 ++
 .../spark/graphx/util/BytecodeUtilsSuite.scala  |   93 ++
 pom.xml                                         |    5 +-
 project/SparkBuild.scala                        |   14 +-
 76 files changed, 7132 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4a805aff/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4a805aff/project/SparkBuild.scala
----------------------------------------------------------------------


[42/50] git commit: Write Graph Builders section in guide

Posted by pw...@apache.org.
Write Graph Builders section in guide


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/67795dbb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/67795dbb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/67795dbb

Branch: refs/heads/master
Commit: 67795dbbfb3857e9677e3104b8bd6fd2cd5633a9
Parents: e14a14b
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 21:45:11 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:45:11 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 54 +++++++++++++++++++++++++++++++----
 1 file changed, 49 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/67795dbb/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 69cadc1..aadeb38 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -673,13 +673,57 @@ println(sssp.vertices.collect.mkString("\n"))
 # Graph Builders
 <a name="graph_builders"></a>
 
-[`GraphLoader.edgeListFile`][GraphLoader.edgeListFile]
+GraphX provides several ways of building a graph from a collection of vertices and edges in an RDD or on disk. None of the graph builders repartitions the graph's edges by default; instead, edges are left in their default partitions (such as their original blocks in HDFS). [`Graph.groupEdges`][Graph.groupEdges] requires the graph to be repartitioned because it assumes identical edges will be colocated on the same partition, so you must call [`Graph.partitionBy`][Graph.partitionBy] before calling `groupEdges`.
 
-[`Graph.apply`][Graph.apply]
+{% highlight scala %}
+object GraphLoader {
+  def edgeListFile(
+      sc: SparkContext,
+      path: String,
+      canonicalOrientation: Boolean = false,
+      minEdgePartitions: Int = 1)
+    : Graph[Int, Int]
+}
+{% endhighlight %}
+
+[`GraphLoader.edgeListFile`][GraphLoader.edgeListFile] provides a way to load a graph from a list of edges on disk. It parses an adjacency list of (source vertex ID, destination vertex ID) pairs of the following form, skipping comment lines that begin with `#`:
+
+~~~
+# This is a comment
+2 1
+4 1
+1 2
+~~~
+
+It creates a `Graph` from the specified edges, automatically creating any vertices mentioned by edges. All vertex and edge attributes default to 1. The `canonicalOrientation` argument allows reorienting edges in the positive direction (`srcId < dstId`), which is required by the [connected components][ConnectedComponents] algorithm. The `minEdgePartitions` argument specifies the minimum number of edge partitions to generate; there may be more edge partitions than specified if, for example, the HDFS file has more blocks.
+
+{% highlight scala %}
+object Graph {
+  def apply[VD, ED](
+      vertices: RDD[(VertexID, VD)],
+      edges: RDD[Edge[ED]],
+      defaultVertexAttr: VD = null)
+    : Graph[VD, ED]
+
+  def fromEdges[VD, ED](
+      edges: RDD[Edge[ED]],
+      defaultValue: VD): Graph[VD, ED]
+
+  def fromEdgeTuples[VD](
+      rawEdges: RDD[(VertexID, VertexID)],
+      defaultValue: VD,
+      uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int]
+
+}
+{% endhighlight %}
+
+[`Graph.apply`][Graph.apply] allows creating a graph from RDDs of vertices and edges. Duplicate vertices are picked arbitrarily and vertices found in the edge RDD but not the vertex RDD are assigned the default attribute.
+
+[`Graph.fromEdges`][Graph.fromEdges] allows creating a graph from only an RDD of edges, automatically creating any vertices mentioned by edges and assigning them the default value.
 
-[`Graph.fromEdgeTuples`][Graph.fromEdgeTuples]
+[`Graph.fromEdgeTuples`][Graph.fromEdgeTuples] allows creating a graph from only an RDD of edge tuples, assigning the edges the value 1, and automatically creating any vertices mentioned by edges and assigning them the default value. It also supports deduplicating the edges; to deduplicate, pass `Some` of a [`PartitionStrategy`][PartitionStrategy] as the `uniqueEdges` parameter (for example, `uniqueEdges = Some(PartitionStrategy.RandomVertexCut)`). A partition strategy is necessary to colocate identical edges on the same partition so they can be deduplicated.
 
-[`Graph.fromEdges`][Graph.fromEdges]
+[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy$
 
 [GraphLoader.edgeListFile]: api/graphx/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int]
 [Graph.apply]: api/graphx/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexID,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED]
@@ -826,7 +870,7 @@ println(ccByUsername.collect().mkString("\n"))
 
 ## Triangle Counting
 
-A vertex is part of a triangle when it has two adjacent vertices with an edge between them. GraphX implements a triangle counting algorithm in the [`TriangleCount` object][TriangleCount] that determines the number of triangles passing through each vertex, providing a measure of clustering. We compute the triangle count of the social network dataset from the [PageRank section](#pagerank). *Note that `TriangleCount` requires the edges to be in canonical orientation (`srcId < dstId`) and the graph to be partitioned using [`Graph#partitionBy`][Graph.partitionBy].*
+A vertex is part of a triangle when it has two adjacent vertices with an edge between them. GraphX implements a triangle counting algorithm in the [`TriangleCount` object][TriangleCount] that determines the number of triangles passing through each vertex, providing a measure of clustering. We compute the triangle count of the social network dataset from the [PageRank section](#pagerank). *Note that `TriangleCount` requires the edges to be in canonical orientation (`srcId < dstId`) and the graph to be partitioned using [`Graph.partitionBy`][Graph.partitionBy].*
 
 [TriangleCount]: api/graphx/index.html#org.apache.spark.graphx.lib.TriangleCount$
 [Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph@partitionBy(PartitionStrategy):Graph[VD,ED]


[20/50] git commit: Merge branch 'scaladoc1' of github.com:rxin/incubator-spark into graphx

Posted by pw...@apache.org.
Merge branch 'scaladoc1' of github.com:rxin/incubator-spark into graphx


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/dc041cd3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/dc041cd3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/dc041cd3

Branch: refs/heads/master
Commit: dc041cd3b6b3b75df42d9a74dcf95179a25ee50f
Parents: e2d25d2 30328c3
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 16:25:21 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 16:25:21 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/streaming/api/java/JavaStreamingContext.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[02/50] git commit: Correcting typos in documentation.

Posted by pw...@apache.org.
Correcting typos in documentation.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/cf57b1b0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/cf57b1b0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/cf57b1b0

Branch: refs/heads/master
Commit: cf57b1b0555b89953f1eb2a2d9819e20fcd17708
Parents: 64c4593
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Sat Jan 11 17:13:10 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Sat Jan 11 17:13:10 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 145 ++++++++++++++++++----------------
 1 file changed, 79 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cf57b1b0/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 5c9f196..9a7c4ac 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -19,11 +19,11 @@ title: GraphX Programming Guide
 GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high-level,
 GraphX extends the Spark [RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the
 [Resilient Distributed property Graph (RDG)](#property_graph): a directed multigraph with properties
-attached to each vertex and edge.  To support graph computation, GraphX exposes a set of functions
-(e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and
-[mapReduceTriplets](#mrTriplets)) as well as an optimized variant of the
-[Pregel](#pregel) API. In addition, GraphX includes a growing collection of graph
-[algorithms](#graph_algorithms) and [builders](#graph_builders) to simplify graph analytics tasks.
+attached to each vertex and edge.  To support graph computation, GraphX exposes a set of fundamental
+operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and
+[mapReduceTriplets](#mrTriplets)) as well as an optimized variant of the [Pregel](#pregel) API. In
+addition, GraphX includes a growing collection of graph [algorithms](#graph_algorithms) and
+[builders](#graph_builders) to simplify graph analytics tasks.
 
 ## Background on Graph-Parallel Computation
 
@@ -65,15 +65,13 @@ in graph-parallel systems, GraphX is able to optimize the execution of graph ope
 
 ## GraphX Replaces the Spark Bagel API
 
-Prior to the release of GraphX, graph computation in Spark was expressed using
-Bagel, an implementation of the Pregel API.  GraphX improves upon Bagel by
-exposing a richer property graph API, a more streamlined version of the Pregel
-abstraction, and system optimizations to improve performance and reduce memory
-overhead.  While we plan to eventually deprecate the Bagel, we will continue to
-support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and
-[Bagel programming guide](bagel-programming-guide.html). However, we encourage
-Bagel users to explore the new GraphX API and comment on issues that may
-complicate the transition from Bagel.
+Prior to the release of GraphX, graph computation in Spark was expressed using Bagel, an
+implementation of Pregel.  GraphX improves upon Bagel by exposing a richer property graph API, a
+more streamlined version of the Pregel abstraction, and system optimizations to improve performance
+and reduce memory overhead.  While we plan to eventually deprecate the Bagel, we will continue to
+support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and [Bagel programming
+guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX
+API and comment on issues that may complicate the transition from Bagel.
 
 # Getting Started
 
@@ -94,41 +92,55 @@ The [property graph](api/graphx/index.html#org.apache.spark.graphx.Graph) is a d
 graph with user defined objects attached to each vertex and edge.  A directed multigraph is a
 directed graph with potentially multiple parallel edges sharing the same source and destination
 vertex.  The ability to support parallel edges simplifies modeling scenarios where there can be
-multiple relationships (e.g., co-worker and friend) between the same vertices.  Note, however there
-can only be one instance of each vertex.
-
-Like RDDs, property graphs are immutable, distributed, and fault-tolerant. Vertices are keyed by
-their vertex identifier (`VertexId`) which is a unique 64-bit long. Similarly, edges have
-corresponding source and destination vertex identifiers. GraphX does not impose any ordering or
-constraints on the vertex identifiers.
-
-The property graph is parameterized over the vertex `VD` and edge `ED` types.  These are the types
-of the objects associated with each vertex and edge respectively.  In some cases it can be desirable
-to have vertices of different types.  However, this can be accomplished through inheritance.
+multiple relationships (e.g., co-worker and friend) between the same vertices.  Each vertex is keyed
+by a *unique* 64-bit long identifier (`VertexId`).  Similarly, edges have corresponding source and
+destination vertex identifiers. GraphX does not impose any ordering or constraints on the vertex
+identifiers.  The property graph is parameterized over the vertex `VD` and edge `ED` types.  These
+are the types of the objects associated with each vertex and edge respectively.
 
 > GraphX optimizes the representation of `VD` and `ED` when they are plain old data-types (e.g.,
 > int, double, etc...) reducing the in memory footprint.
 
+In some cases we may wish to have vertices with different property types in the same graph. This can
+be accomplished through inheritance.  For example to model users and products as a bipartie graph we
+might do the following:
+
+{% highlight scala %}
+case class VertexProperty
+case class UserProperty extends VertexProperty
+  (val name: String)
+case class ProductProperty extends VertexProperty
+  (val name: String, val price: Double)
+// The graph might then have the type:
+val graph: Graph[VertexProperty, String]
+{% endhighlight %}
+
+Like RDDs, property graphs are immutable, distributed, and fault-tolerant.  Changes to the values or
+structure of the graph are accomplished by producing a new graph with the desired changes. The graph
+is partitioned across the workers using a range of vertex-partitioning heuristics.  As with RDDs,
+each partition of the graph can be recreated on a different machine in the event of a failure.
+
 Logically the property graph corresponds to a pair of typed collections (RDDs) encoding the
-properties for each vertex and edge:
+properties for each vertex and edge.  As a consequence, the graph class contains members to access
+the vertices and edges of the graph:
 
 {% highlight scala %}
-class Graph[VD: ClassTag, ED: ClassTag] {
-  val vertices: RDD[(VertexId, VD)]
-  val edges: RDD[Edge[ED]]
-  // ...
-}
+val vertices: VertexRDD[VD]
+val edges: EdgeRDD[ED]
 {% endhighlight %}
 
-> Note that the vertices and edges of the graph are actually of type `VertexRDD[VD]` and
-> `EdgeRDD[ED]` respectively. These classes extend and are optimized versions of `RDD[(VertexId,
-> VD)]` and `RDD[Edge[ED]]` with additional functionality built around the internal index and column
-> oriented representations.  We discuss the `VertexRDD` and `EdgeRDD` API in greater detail in the
-> section on [vertex and edge RDDs](#vertex_and_edge_rdds)
+The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexId,
+VD)]` and `RDD[Edge[ED]]` respectively.  Both `VertexRDD[VD]` and `EdgeRDD[ED]` provide  additional
+functionality built around graph computation and leverage internal optimizations.  We discuss the
+`VertexRDD` and `EdgeRDD` API in greater detail in the section on [vertex and edge
+RDDs](#vertex_and_edge_rdds) but for now they can be thought of as simply RDDs of the form:
+`RDD[(VertexId, VD)]` and `RDD[Edge[ED]]`.
+
+### Example Property Graph
 
-For example, we might construct a property graph consisting of various collaborators on the GraphX
-project. The vertex property contains the username and occupation and the edge property contains
-a string describing the relationships between the users.
+Suppose we want to construct a property graph consisting of the various collaborators on the GraphX
+project. The vertex property might contain the username and occupation.  We could annotate edges
+with a string describing the relationships between collaborators:
 
 <p style="text-align: center;">
   <img src="img/property_graph.png"
@@ -183,18 +195,19 @@ graph.vertices.filter { case (id, (name, pos)) => pos == "postdoc"}.count
 graph.edges.filter(e => e.srcId > e.dstId).count
 {% endhighlight %}
 
-> Note that `graph.vertices` returns an `RDD[(VertexId, (String, String))]` and so we must use the
-> scala `case` expression to deconstruct the tuple.  Alternatively, `graph.edges` returns an `RDD`
-> containing `Edge[String]` objects.  We could have also used the case class type constructor as
-> in the following:
+> Note that `graph.vertices` returns an `VertexRDD[(String, String)]` which extends
+> `RDD[(VertexId, (String, String))]` and so we use the scala `case` expression to deconstruct
+> the tuple.  Alternatively, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects.
+> We could have also used the case class type constructor as in the following:
 > {% highlight scala %}
 graph.edges.filter { case Edge(src, dst, prop) => src < dst }.count
 {% endhighlight %}
 
 In addition to the vertex and edge views of the property graph, GraphX also exposes a triplet view.
 The triplet view logically joins the vertex and edge properties yielding an `RDD[EdgeTriplet[VD,
-ED]]` consisting of [`EdgeTriplet`](api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet).
-This *join* can be expressed in the following SQL expression:
+ED]]` containing instances of the
+[`EdgeTriplet`](api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet) class. This *join* can be
+expressed in the following SQL expression:
 
 {% highlight sql %}
 SELECT src.id, dst.id, src.attr, e.attr, dst.attr
@@ -266,7 +279,7 @@ defined `map` function.
 > does not preserve the structural indicies and would not benefit from the substantial system
 > optimizations in GraphX.
 > {% highlight scala %}
-val newVertices = graph.vertices.map { case (id, attr) => (id, mapUdf(id, attr))}
+val newVertices = graph.vertices.map { case (id, attr) => (id, mapUdf(id, attr)) }
 val newGraph = Graph(newVertices, graph.edges)
 {% endhighlight %}
 
@@ -291,12 +304,9 @@ add more in the future.  The following is a list of the basic structural operato
 
 {% highlight scala %}
 def reverse: Graph[VD, ED]
-
-def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true),
-  vpred: (VertexID, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED]
-
+def subgraph(epred: EdgeTriplet[VD,ED] => Boolean,
+             vpred: (VertexID, VD) => Boolean): Graph[VD, ED]
 def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED]
-
 def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED]
 {% endhighlight %}
 
@@ -309,7 +319,7 @@ The `subgraph` operator takes vertex and edge predicates and returns the graph c
 vertices that satisfy the vertex predicate (evaluate to true) and edges that satisfy the edge
 predicate *and connect vertices that satisfy the vertex predicate*.  The `subgraph` operator can be
 used in number of situations to restrict the graph to the vertices and edges of interest or
-eliminate broken links.  For example in the following code we remove broken links:
+eliminate broken links. For example in the following code we remove broken links:
 
 {% highlight scala %}
 val users: RDD[(VertexId, (String, String))]
@@ -322,32 +332,35 @@ val graph = Graph(users, relationships, defaultUser)
 val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing")
 {% endhighlight %}
 
-The `mask` operators returns the subgraph containing only the vertices and edges that are found in
-the input graph.  This can be used in conjunction with the `subgraph` operator to restrict a graph
-based on the properties in another related graph.  For example, we might run connected components
-using the graph with missing vertices and then restrict the answer to the valid subgraph.
+> Note in the above example only the vertex predicate is provided.  The `subgraph` operator defaults
+> to `true` if the vertex or edge predicates are not provided.
+
+The `mask` operator also constructs a subgraph by returning a graph that contains the vertices and
+edges that are also found in the input graph.  This can be used in conjunction with the `subgraph`
+operator to restrict a graph based on the properties in another related graph.  For example, we
+might run connected components using the graph with missing vertices and then restrict the answer to
+the valid subgraph.
 
 {% highlight scala %}
 // Run Connected Components
-val ccGraph = graph.connectedComponents()
+val ccGraph = graph.connectedComponents() // No longer contains missing field
 // Remove missing vertices as well as the edges to connected to them
 val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing")
 // Restrict the answer to the valid subgraph
 val validCCGraph = ccGraph.mask(validGraph)
 {% endhighlight %}
 
-The `groupEdges` operator merges parallel edges: duplicate edges between pairs of vertices.  In many
-numerical applications parallel edges can be *added* (their weights combined) into a single edge
-thereby reducing the graph size in memory as well as the cost of computation.
+The `groupEdges` operator merges parallel edges (i.e., duplicate edges between pairs of vertices) in
+the multigraph.  In many numerical applications, parallel edges can be *added* (their weights
+combined) into a single edge thereby reducing the size of the graph.
 
 ## Join Operators
 <a name="join_operators"></a>
 
-The ability to move between graph and collection views is a key part of GraphX.  In many cases it is
-necessary to join data from external collections (RDDs) with graphs.  For example, we might have
-extra user properties that we want to merge with an existing graph or we might want to pull vertex
-properties from one graph into another.  These tasks can be accomplished using the *join* operators.
-Below we list the key join operators:
+In many cases it is necessary to join data from external collections (RDDs) with graphs.  For
+example, we might have extra user properties that we want to merge with an existing graph or we
+might want to pull vertex properties from one graph into another.  These tasks can be accomplished
+using the *join* operators. Below we list the key join operators:
 
 {% highlight scala %}
 def joinVertices[U](table: RDD[(VertexID, U)])(map: (VertexID, VD, U) => VD)
@@ -356,7 +369,7 @@ def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(map: (VertexID, VD, Opt
   : Graph[VD2, ED]
 {% endhighlight %}
 
-The `joinVertices` operators, defined in
+The `joinVertices` operator, defined in
 [`GraphOps.scala`](api/graphx/index.html#org.apache.spark.graphx.GraphOps), joins the vertices with
 the input RDD and returns a new graph with the vertex properties obtained by applying the user
 defined `map` function to the result of the joined vertices.  Vertices without a matching value in


[34/50] git commit: Merge branch 'graphx' of github.com:ankurdave/incubator-spark into graphx

Posted by pw...@apache.org.
Merge branch 'graphx' of github.com:ankurdave/incubator-spark into graphx


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0fbc0b05
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0fbc0b05
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0fbc0b05

Branch: refs/heads/master
Commit: 0fbc0b056179ec85b86d7ad6ddcd379bf1bf194d
Parents: 0b18bfb 552de5d
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 18:51:22 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 18:51:22 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 50 ++++++++++++++++++++++++-----------
 1 file changed, 35 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[06/50] git commit: Documenting Pregel API

Posted by pw...@apache.org.
Documenting Pregel API


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c787ff56
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c787ff56
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c787ff56

Branch: refs/heads/master
Commit: c787ff5640ad9d6f6dc3b744d73a1cb0c91eb90a
Parents: 7a4bb86
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Sun Jan 12 20:49:41 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Sun Jan 12 20:49:52 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 199 +++++++++++++++++++++++++++++++++-
 1 file changed, 198 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c787ff56/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 22feccb..8975941 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -429,12 +429,209 @@ val joinedGraph = graph.joinVertices(uniqueCosts,
 {% endhighlight %}
 
 
-## Map Reduce Triplets (mapReduceTriplets)
+## Neighborhood Aggregation
+
+A key part of graph computation is aggregating information about the neighborhood of each vertex.
+For example we might want to know the number of followers each user has or the average age of the
+the followers of each user.  Many iterative graph algorithms (e.g., PageRank, Shortest Path, and
+connected components) repeatedly aggregate properties of neighboring vertices (e.g., current
+PageRank Value, shortest path to the source, and smallest reachable vertex id).
+
+### Map Reduce Triplets (mapReduceTriplets)
 <a name="mrTriplets"></a>
 
+[Graph.mapReduceTriplets]: api/graphx/index.html#mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=&gt;Iterator[(org.apache.spark.graphx.VertexID,A)],reduceFunc:(A,A)=&gt;A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A]
+
+These core (heavily optimized) aggregation primitive in GraphX is the
+(`mapReduceTriplets`)[Graph.mapReduceTriplets] operator:
+
+{% highlight scala %}
+def mapReduceTriplets[A](
+    map: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+    reduce: (A, A) => A)
+  : VertexRDD[A]
+{% endhighlight %}
+
+The (`mapReduceTriplets`)[Graph.mapReduceTriplets] operator takes a user defined map function which
+is applied to each triplet and can yield *messages* destined to either (none or both) vertices in
+the triplet.  We currently only support messages destined to the source or destination vertex of the
+triplet to enable optimized preaggregation.  The user defined `reduce` function combines the
+messages destined to each vertex.  The `mapReduceTriplets` operator returns a `VertexRDD[A]`
+containing the aggregate message to each vertex.  Vertices that do not receive a message are not
+included in the returned `VertexRDD`.
+
+> Note that `mapReduceTriplets takes an additional optional `activeSet` (see API docs) which
+> restricts the map phase to edges adjacent to the vertices in the provided `VertexRDD`. Restricting
+> computation to triplets adjacent to a subset of the vertices is often necessary in incremental
+> iterative computation and is a key part of the GraphX implementation of Pregel.
+
+We can use the `mapReduceTriplets` operator to collect information about adjacent vertices.  For
+example if we wanted to compute the average age of followers who are older that each user we could
+do the following.
+
+{% highlight scala %}
+// Graph with age as the vertex property
+val graph: Graph[Double, String] = getFromSomewhereElse()
+// Compute the number of older followers and their total age
+val olderFollowers: VertexRDD[(Int, Double)] = graph.mapReduceTriplets[(Int, Double)](
+  triplet => { // Map Function
+    if (triplet.srcAttr > triplet.dstAttr) {
+      // Send message to destination vertex containing counter and age
+      Iterator((triplet.dstId, (1, triplet.srcAttr)))
+    } else {
+      // Don't send a message for this triplet
+      Iterator.empty
+    }
+  },
+  // Add counter and age
+  (a, b) => (a._1 + b._1, a._2 + b._2) // Reduce Function
+)
+// Divide total age by number of older followers to get average age of older followers
+val avgAgeOlderFollowers: VertexRDD[Double] =
+  olderFollowers.mapValues { case (count, totalAge) => totalAge / count }
+{% endhighlight %}
+
+> Note that the `mapReduceTriplets` operation performs optimally when the messages (and their sums)
+> are constant sized (e.g., floats and addition instead of lists and concatenation).  More
+> precisely, the result of `mapReduceTriplets` should be sub-linear in the degree of each vertex.
+
+Because it is often necessary to aggregate information about neighboring vertices we also provide an
+alternative interface defined in [`GraphOps`][GraphOps]:
+
+{% highlight scala %}
+def aggregateNeighbors[A](
+    map: (VertexID, EdgeTriplet[VD, ED]) => Option[A],
+    reduce: (A, A) => A,
+    edgeDir: EdgeDirection)
+  : VertexRDD[A]
+{% endhighlight %}
+
+The `aggregateNeighbors` operator is implemented directly on top of `mapReduceTriplets` but allows
+the user to define the logic in a more vertex centric manner.  Here the `map` function is provided
+the vertex to which the message is sent as well as one of the edges and returns the optional message
+value.  The `edgeDir` determines whether the `map` function is run on `In`, `Out`, or `All` edges
+adjacent to each vertex.
+
+### Computing Degree Information
+
+A common aggregation task is computing the degree of each vertex: the number of edges adjacent to
+each vertex.  In the context of directed graphs it often necessary to know the in-degree, out-
+degree, and the total degree of each vertex.  The  [`GraphOps`][GraphOps] class contains a
+collection of operators to compute the degrees of each vertex.  For example in the following we
+compute the max in, out, and total degrees:
+
+{% highlight scala %}
+// Define a reduce operation to compute the highest degree vertex
+def maxReduce(a: (VertexId, Int), b: (VertexId, Int)): (VertexId, Int) = {
+  if (a._2 > b._2) a else b
+}
+// Compute the max degrees
+val maxInDegree: (VertexId, Int)  = graph.inDegrees.reduce(maxReduce)
+val maxOutDegree: (VertexId, Int) = graph.outDegrees.reduce(maxReduce)
+val maxDegrees: (VertexId, Int)   = graph.degrees.reduce(maxReduce)
+{% endhighlight %}
+
+
+### Collecting Neighbors
+
+In some cases it may be easier to express computation by collecting neighboring vertices and their
+attributes at each vertex. This can be easily accomplished using the `collectNeighborIds` and the
+`collectNeighbors` operators.
+
+{% highlight scala %}
+def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]] =
+def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[ Array[(VertexID, VD)] ]
+{% endhighlight %}
+
+> Note that these operators can be quite costly as they duplicate information and require
+> substantial communication.  If possible try expressing the same computation using the
+> `mapReduceTriplets` operator directly.
+
 # Pregel API
 <a name="pregel"></a>
 
+Graphs are inherently recursive data-structures as properties of a vertices depend on properties of
+their neighbors which intern depend on properties of the neighbors of their neighbors.  As a
+consequence many important graph algorithms iteratively recompute the properties of each vertex
+until a fixed-point condition is reached.  A range of graph-parallel abstractions have been proposed
+to express these iterative algorithms.  GraphX exposes a Pregel operator which is a fusion of
+the widely used Pregel and GraphLab abstractions.
+
+At a high-level the GraphX variant of the Pregel abstraction is a bulk-synchronous parallel
+messaging abstraction constrained to the topology of the graph.  The Pregel operator executes in a
+series of super-steps in which vertices receive the sum of their inbound messages from the previous
+super-step, compute a new property value, and then send messages to neighboring vertices in the next
+super-step.  Vertices that do not receive a message are skipped within a super-step.  The Pregel
+operators terminates iteration and returns the final graph when there are no messages remaining.
+
+> Note, unlike more standard Pregel implementations, vertices in GraphX can only send messages to
+> neighboring vertices and the message construction is done in parallel using a user defined
+> messaging function.  These constraints allow additional optimization within GraphX.
+
+The following is type signature of the Pregel operator as well as a *sketch* of its implementation
+(note calls to graph.cache have been removed):
+
+{% highlight scala %}
+def pregel[A]
+    (initialMsg: A,
+     maxIter: Int = Int.MaxValue,
+     activeDir: EdgeDirection = EdgeDirection.Out)
+    (vprog: (VertexID, VD, A) => VD,
+     sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
+     mergeMsg: (A, A) => A)
+  : Graph[VD, ED] = {
+  // Receive the initial message at each vertex
+  var g = mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache()
+  // compute the messages
+  var messages = g.mapReduceTriplets(sendMsg, mergeMsg)
+  var activeMessages = messages.count()
+  // Loop until no messages remain or maxIterations is achieved
+  var i = 0
+  while (activeMessages > 0 && i < maxIterations) {
+    // Receive the messages: -----------------------------------------------------------------------
+    // Run the vertex program on all vertices that receive messages
+    val newVerts = g.vertices.innerJoin(messages)(vprog).cache()
+    // Merge the new vertex values back into the graph
+    g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) }.cache()
+    // Send Messages: ------------------------------------------------------------------------------
+    // Vertices that didn't receive a message above don't appear in newVerts and therefore don't
+    // get to send messages.  More precisely the map phase of mapReduceTriplets is only invoked
+    // on edges in the activeDir of vertices in newVerts
+    messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, activeDir))).cache()
+    activeMessages = messages.count()
+    i += 1
+  }
+  g
+}
+{% endhighlight %}
+
+Notice that Pregel takes two argument lists (i.e., `graph.pregel(list1)(list2)`).  The first
+argument list contains configuration parameters including the initial message, the maximum number of
+iterations, and the edge direction in which to send messages (by default along out edges).  The
+second argument list contains the user defined functions for receiving messages (the vertex program
+`vprog`), computing messages (`sendMsg`), and combining messages `mergeMsg`.
+
+We can use the Pregel operator to express computation such single source shortest path in the
+following example.
+
+{% highlight scala %}
+val graph: Graph[String, Double] // A graph with edge attributes containing distances
+val sourceId: VertexId = 42 // The ultimate source
+// Initialize the graph such that all vertices except the root have distance infinity.
+val initialGraph = graph.mapVertices((id, _) => if (id == shourceId) 0.0 else Double.PositiveInfinity)
+val sssp = initialGraph.pregel(Double.PositiveInfinity)(
+  (id, dist, newDist) => math.min(dist, newDist) // Vertex Program
+  triplet => {  // Send Message
+    if(triplet.srcAttr + triplet.attr < triplet.dstAttr) {
+      Iterator((triplet.dstId, triplet.srcAttr + triplet.attr))
+    } else {
+      Iterator.empty
+    }
+  },
+  (a,b) => math.min(a,b) // Merge Message
+  )
+{% endhighlight %}
+
 # Graph Builders
 <a name="graph_builders"></a>
 


[32/50] git commit: More cleanup.

Posted by pw...@apache.org.
More cleanup.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/9317286b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/9317286b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/9317286b

Branch: refs/heads/master
Commit: 9317286b72ec8bb065b0422c344c267cc49189e3
Parents: 8e5c732
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 18:45:35 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 18:45:35 2014 -0800

----------------------------------------------------------------------
 graphx/src/main/scala/org/apache/spark/graphx/Edge.scala  |  2 +-
 .../scala/org/apache/spark/graphx/EdgeDirection.scala     |  3 +++
 .../org/apache/spark/graphx/lib/ConnectedComponents.scala |  4 +---
 .../scala/org/apache/spark/graphx/lib/TriangleCount.scala | 10 +++++-----
 4 files changed, 10 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9317286b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
index 8546305..21be58e 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
@@ -37,7 +37,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED]
     if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In }
 }
 
-object Edge {
+private[graphx] object Edge {
   def lexicographicOrdering[ED] = new Ordering[Edge[ED]] {
     override def compare(a: Edge[ED], b: Edge[ED]): Int =
       (if (a.srcId != b.srcId) a.srcId - b.srcId else a.dstId - b.dstId).toInt

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9317286b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
index 5b58a61..f265764 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
@@ -26,6 +26,9 @@ class EdgeDirection private (private val name: String) extends Serializable {
 }
 
 
+/**
+ * A set of [[EdgeDirection]]s.
+ */
 object EdgeDirection {
   /** Edges arriving at a vertex. */
   final val In = new EdgeDirection("In")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9317286b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
index 121df5a..4d1f5e7 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
@@ -14,13 +14,11 @@ object ConnectedComponents {
    * @tparam ED the edge attribute type (preserved in the computation)
    *
    * @param graph the graph for which to compute the connected components
-   * @param undirected compute reachability ignoring edge direction.
    *
    * @return a graph with vertex attributes containing the smallest vertex in each
    *         connected component
    */
-  def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]):
-    Graph[VertexID, ED] = {
+  def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = {
     val ccGraph = graph.mapVertices { case (vid, _) => vid }
     def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
       if (edge.srcAttr < edge.dstAttr) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9317286b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
index 4b04557..23c9c40 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
@@ -9,11 +9,11 @@ import org.apache.spark.graphx._
  *
  * The algorithm is relatively straightforward and can be computed in three steps:
  *
- * 1) Compute the set of neighbors for each vertex
- * 2) For each edge compute the intersection of the sets and send the
- *    count to both vertices.
- * 3) Compute the sum at each vertex and divide by two since each
- *    triangle is counted twice.
+ * <ul>
+ * <li>Compute the set of neighbors for each vertex
+ * <li>For each edge compute the intersection of the sets and send the count to both vertices.
+ * <li> Compute the sum at each vertex and divide by two since each triangle is counted twice.
+ * </ul>
  *
  * Note that the input graph should have its edges in canonical direction
  * (i.e. the `sourceId` less than `destId`). Also the graph must have been partitioned


[22/50] git commit: Add EdgeDirection.Either and use it to fix CC bug

Posted by pw...@apache.org.
Add EdgeDirection.Either and use it to fix CC bug

The bug was due to a misunderstanding of the activeSetOpt parameter to
Graph.mapReduceTriplets. Passing EdgeDirection.Both causes
mapReduceTriplets to run only on edges with *both* vertices in the
active set. This commit adds EdgeDirection.Either, which causes
mapReduceTriplets to run on edges with *either* vertex in the active
set. This is what connected components needed.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ae4b75d9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ae4b75d9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ae4b75d9

Branch: refs/heads/master
Commit: ae4b75d94a4a0f2545e6d90d6f9b8f162bf70ded
Parents: 1bd5cef
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 16:48:11 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 17:03:03 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/graphx/EdgeDirection.scala |  8 +++-
 .../scala/org/apache/spark/graphx/Graph.scala   |  9 +++--
 .../org/apache/spark/graphx/GraphOps.scala      | 33 ++++++++++------
 .../scala/org/apache/spark/graphx/Pregel.scala  |  7 +++-
 .../apache/spark/graphx/impl/GraphImpl.scala    |  4 ++
 .../spark/graphx/lib/ConnectedComponents.scala  | 41 +++++++-------------
 .../org/apache/spark/graphx/lib/PageRank.scala  |  5 ++-
 .../lib/StronglyConnectedComponents.scala       |  3 +-
 .../apache/spark/graphx/lib/TriangleCount.scala |  2 +-
 .../org/apache/spark/graphx/GraphOpsSuite.scala |  2 +-
 .../org/apache/spark/graphx/PregelSuite.scala   |  2 +-
 .../graphx/lib/ConnectedComponentsSuite.scala   |  2 +-
 12 files changed, 64 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
index 9d37f65..5b58a61 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
@@ -6,11 +6,12 @@ package org.apache.spark.graphx
 class EdgeDirection private (private val name: String) extends Serializable {
   /**
    * Reverse the direction of an edge.  An in becomes out,
-   * out becomes in and both remains both.
+   * out becomes in and both and either remain the same.
    */
   def reverse: EdgeDirection = this match {
     case EdgeDirection.In   => EdgeDirection.Out
     case EdgeDirection.Out  => EdgeDirection.In
+    case EdgeDirection.Either => EdgeDirection.Either
     case EdgeDirection.Both => EdgeDirection.Both
   }
 
@@ -32,6 +33,9 @@ object EdgeDirection {
   /** Edges originating from a vertex. */
   final val Out = new EdgeDirection("Out")
 
-  /** All edges adjacent to a vertex. */
+  /** Edges originating from *or* arriving at a vertex of interest. */
+  final val Either = new EdgeDirection("Either")
+
+  /** Edges originating from *and* arriving at a vertex of interest. */
   final val Both = new EdgeDirection("Both")
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 7d4f0de..49705fd 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -274,9 +274,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
    * of the map phase
    *
    * @param activeSetOpt optionally, a set of "active" vertices and a direction of edges to consider
-   * when running `mapFunc`. For example, if the direction is Out, `mapFunc` will only be run on
-   * edges originating from vertices in the active set. The active set must have the same index as
-   * the graph's vertices.
+   * when running `mapFunc`. If the direction is `In`, `mapFunc` will only be run on edges with
+   * destination in the active set.  If the direction is `Out`, `mapFunc` will only be run on edges
+   * originating from vertices in the active set. If the direction is `Either`, `mapFunc` will be
+   * run on edges with *either* vertex in the active set. If the direction is `Both`, `mapFunc` will
+   * be run on edges with *both* vertices in the active set. The active set must have the same index
+   * as the graph's vertices.
    *
    * @example We can use this function to compute the in-degree of each
    * vertex

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 578eb33..66d5180 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -38,7 +38,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    * The degree of each vertex in the graph.
    * @note Vertices with no edges are not returned in the resulting RDD.
    */
-  lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Both)
+  lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either)
 
   /**
    * Computes the neighboring vertex degrees.
@@ -50,7 +50,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
       graph.mapReduceTriplets(et => Iterator((et.dstId,1)), _ + _)
     } else if (edgeDirection == EdgeDirection.Out) {
       graph.mapReduceTriplets(et => Iterator((et.srcId,1)), _ + _)
-    } else { // EdgeDirection.both
+    } else { // EdgeDirection.Either
       graph.mapReduceTriplets(et => Iterator((et.srcId,1), (et.dstId,1)), _ + _)
     }
   }
@@ -65,7 +65,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    */
   def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]] = {
     val nbrs =
-      if (edgeDirection == EdgeDirection.Both) {
+      if (edgeDirection == EdgeDirection.Either) {
         graph.mapReduceTriplets[Array[VertexID]](
           mapFunc = et => Iterator((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))),
           reduceFunc = _ ++ _
@@ -79,7 +79,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
           mapFunc = et => Iterator((et.dstId, Array(et.srcId))),
           reduceFunc = _ ++ _)
       } else {
-        throw new SparkException("It doesn't make sense to collect neighbor ids without a direction.")
+        throw new SparkException("It doesn't make sense to collect neighbor ids without a " +
+          "direction. (EdgeDirection.Both is not supported; use EdgeDirection.Either instead.)")
       }
     graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) =>
       nbrsOpt.getOrElse(Array.empty[VertexID])
@@ -100,11 +101,19 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    */
   def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexID, VD)]] = {
     val nbrs = graph.mapReduceTriplets[Array[(VertexID,VD)]](
-      edge => Iterator(
-        (edge.srcId, Array((edge.dstId, edge.dstAttr))),
-        (edge.dstId, Array((edge.srcId, edge.srcAttr)))),
-      (a, b) => a ++ b,
-      edgeDirection)
+      edge => {
+        val msgToSrc = (edge.srcId, Array((edge.dstId, edge.dstAttr)))
+        val msgToDst = (edge.dstId, Array((edge.srcId, edge.srcAttr)))
+        edgeDirection match {
+          case EdgeDirection.Either => Iterator(msgToSrc, msgToDst)
+          case EdgeDirection.In => Iterator(msgToDst)
+          case EdgeDirection.Out => Iterator(msgToSrc)
+          case EdgeDirection.Both =>
+            throw new SparkException("collectNeighbors does not support EdgeDirection.Both. Use" +
+              "EdgeDirection.Either instead.")
+        }
+      },
+      (a, b) => a ++ b)
 
     graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) =>
       nbrsOpt.getOrElse(Array.empty[(VertexID, VD)])
@@ -237,7 +246,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
   def pregel[A: ClassTag](
       initialMsg: A,
       maxIterations: Int = Int.MaxValue,
-      activeDirection: EdgeDirection = EdgeDirection.Out)(
+      activeDirection: EdgeDirection = EdgeDirection.Either)(
       vprog: (VertexID, VD, A) => VD,
       sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
       mergeMsg: (A, A) => A)
@@ -271,8 +280,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    *
    * @see [[org.apache.spark.graphx.lib.ConnectedComponents]]
    */
-  def connectedComponents(undirected: Boolean = true): Graph[VertexID, ED] = {
-    ConnectedComponents.run(graph, undirected)
+  def connectedComponents(): Graph[VertexID, ED] = {
+    ConnectedComponents.run(graph)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
index 83e28d0..75b44dd 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
@@ -67,7 +67,10 @@ object Pregel {
    *
    * @param activeDirection the direction of edges incident to a vertex that received a message in
    * the previous round on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only
-   * out-edges of vertices that received a message in the previous round will run.
+   * out-edges of vertices that received a message in the previous round will run. The default is
+   * `EdgeDirection.Either`, which will run `sendMsg` on edges where either side received a message
+   * in the previous round. If this is `EdgeDirection.Both`, `sendMsg` will only run on edges where
+   * *both* vertices received a message.
    *
    * @param vprog the user-defined vertex program which runs on each
    * vertex and receives the inbound message and computes a new vertex
@@ -90,7 +93,7 @@ object Pregel {
    */
   def apply[VD: ClassTag, ED: ClassTag, A: ClassTag]
     (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue,
-      activeDirection: EdgeDirection = EdgeDirection.Out)(
+      activeDirection: EdgeDirection = EdgeDirection.Either)(
       vprog: (VertexID, VD, A) => VD,
       sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
       mergeMsg: (A, A) => A)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 6a2abc7..c21f893 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -275,6 +275,10 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
           } else {
             edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId))
           }
+        case Some(EdgeDirection.Either) =>
+          // TODO: Because we only have a clustered index on the source vertex ID, we can't filter
+          // the index here. Instead we have to scan all edges and then do the filter.
+          edgePartition.iterator.filter(e => vPart.isActive(e.srcId) || vPart.isActive(e.dstId))
         case Some(EdgeDirection.Out) =>
           if (activeFraction < 0.8) {
             edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
index d078d2a..d057c93 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
@@ -19,37 +19,22 @@ object ConnectedComponents {
    * @return a graph with vertex attributes containing the smallest vertex in each
    *         connected component
    */
-  def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], undirected: Boolean = true):
+  def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]):
     Graph[VertexID, ED] = {
     val ccGraph = graph.mapVertices { case (vid, _) => vid }
-    if (undirected) {
-      def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
-        if (edge.srcAttr < edge.dstAttr) {
-          Iterator((edge.dstId, edge.srcAttr))
-        } else if (edge.srcAttr > edge.dstAttr) {
-          Iterator((edge.srcId, edge.dstAttr))
-        } else {
-          Iterator.empty
-        }
+    def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
+      if (edge.srcAttr < edge.dstAttr) {
+        Iterator((edge.dstId, edge.srcAttr))
+      } else if (edge.srcAttr > edge.dstAttr) {
+        Iterator((edge.srcId, edge.dstAttr))
+      } else {
+        Iterator.empty
       }
-      val initialMessage = Long.MaxValue
-      Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Both)(
-        vprog = (id, attr, msg) => math.min(attr, msg),
-        sendMsg = sendMessage,
-        mergeMsg = (a, b) => math.min(a, b))
-    } else {
-      def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
-        if (edge.srcAttr < edge.dstAttr) {
-          Iterator((edge.dstId, edge.srcAttr))
-        } else {
-          Iterator.empty
-        }
-      }
-      val initialMessage = Long.MaxValue
-      Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Out)(
-        vprog = (id, attr, msg) => math.min(attr, msg),
-        sendMsg = sendMessage,
-        mergeMsg = (a, b) => math.min(a, b))
     }
+    val initialMessage = Long.MaxValue
+    Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Either)(
+      vprog = (id, attr, msg) => math.min(attr, msg),
+      sendMsg = sendMessage,
+      mergeMsg = (a, b) => math.min(a, b))
   } // end of connectedComponents
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
index cf95267..6ced246 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
@@ -77,7 +77,7 @@ object PageRank extends Logging {
     val initialMessage = 0.0
 
     // Execute pregel for a fixed number of iterations.
-    Pregel(pagerankGraph, initialMessage, numIter)(
+    Pregel(pagerankGraph, initialMessage, numIter, activeDirection = EdgeDirection.Out)(
       vertexProgram, sendMessage, messageCombiner)
   }
 
@@ -153,7 +153,8 @@ object PageRank extends Logging {
     val initialMessage = resetProb / (1.0 - resetProb)
 
     // Execute a dynamic version of Pregel.
-    Pregel(pagerankGraph, initialMessage)(vertexProgram, sendMessage, messageCombiner)
+    Pregel(pagerankGraph, initialMessage, activeDirection = EdgeDirection.Out)(
+      vertexProgram, sendMessage, messageCombiner)
       .mapVertices((vid, attr) => attr._1)
   } // end of deltaPageRank
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
index 43c4b9c..edffbcc 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
@@ -53,7 +53,8 @@ object StronglyConnectedComponents {
 
       // collect min of all my neighbor's scc values, update if it's smaller than mine
       // then notify any neighbors with scc values larger than mine
-      sccWorkGraph = Pregel[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Long.MaxValue)(
+      sccWorkGraph = Pregel[(VertexID, Boolean), ED, VertexID](
+        sccWorkGraph, Long.MaxValue, activeDirection = EdgeDirection.Out)(
         (vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2),
         e => {
           if (e.srcId < e.dstId) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
index 58da9e3..d3e22b1 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
@@ -28,7 +28,7 @@ object TriangleCount {
 
     // Construct set representations of the neighborhoods
     val nbrSets: VertexRDD[VertexSet] =
-      g.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) =>
+      g.collectNeighborIds(EdgeDirection.Either).mapValues { (vid, nbrs) =>
         val set = new VertexSet(4)
         var i = 0
         while (i < nbrs.size) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
index 7a90140..280f50e 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
@@ -28,7 +28,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext {
       val chain = (0 until 100).map(x => (x, (x+1)%100) )
       val rawEdges = sc.parallelize(chain, 3).map { case (s,d) => (s.toLong, d.toLong) }
       val graph = Graph.fromEdgeTuples(rawEdges, 1.0).cache()
-      val nbrs = graph.collectNeighborIds(EdgeDirection.Both).cache()
+      val nbrs = graph.collectNeighborIds(EdgeDirection.Either).cache()
       assert(nbrs.count === chain.size)
       assert(graph.numVertices === nbrs.count)
       nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
index 1ff3d75..bceff11 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
@@ -32,7 +32,7 @@ class PregelSuite extends FunSuite with LocalSparkContext {
         Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet)
       val result = Pregel(chainWithSeed, 0)(
         (vid, attr, msg) => math.max(msg, attr),
-        et => Iterator((et.dstId, et.srcAttr)),
+        et => if (et.dstAttr != et.srcAttr) Iterator((et.dstId, et.srcAttr)) else Iterator.empty,
         (a: Int, b: Int) => math.max(a, b))
       assert(result.vertices.collect.toSet ===
         chain.vertices.mapValues { (vid, attr) => attr + 1 }.collect.toSet)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae4b75d9/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
index 86da8f1..27c8705 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
@@ -102,7 +102,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext {
       val defaultUser = ("John Doe", "Missing")
       // Build the initial Graph
       val graph = Graph(users, relationships, defaultUser)
-      val ccGraph = graph.connectedComponents(undirected = true)
+      val ccGraph = graph.connectedComponents()
       val vertices = ccGraph.vertices.collect
       for ( (id, cc) <- vertices ) {
         assert(cc == 0)


[36/50] git commit: Remove Graph.statistics and GraphImpl.printLineage

Posted by pw...@apache.org.
Remove Graph.statistics and GraphImpl.printLineage


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/d4d9ece1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/d4d9ece1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/d4d9ece1

Branch: refs/heads/master
Commit: d4d9ece1af258ccdc83afbb8ba26345d7af16422
Parents: ee8931d
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 18:45:46 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:02:37 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/graphx/Graph.scala   |  7 +--
 .../apache/spark/graphx/impl/GraphImpl.scala    | 65 --------------------
 .../org/apache/spark/graphx/lib/PageRank.scala  |  6 --
 3 files changed, 1 insertion(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d4d9ece1/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 1e3f389..7e99276 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -91,11 +91,6 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
   def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED]
 
   /**
-   * Computes statistics describing the graph representation.
-   */
-  def statistics: Map[String, Any]
-
-  /**
    * Transforms each vertex attribute in the graph using the map function.
    *
    * @note The new graph has the same structure.  As a consequence the underlying index structures
@@ -254,7 +249,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] {
   def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED]
 
   /**
-   * Computes statistics about the neighboring edges and vertices of each vertex.  The user supplied
+   * Aggregates values from the neighboring edges and vertices of each vertex.  The user supplied
    * `mapFunc` function is invoked on each edge of the graph, generating 0 or more "messages" to be
    * "sent" to either vertex in the edge.  The `reduceFunc` is then used to combine the output of
    * the map phase destined to each vertex.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d4d9ece1/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 348490c..12d46a8 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -83,71 +83,6 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
     GraphImpl(vertices, newEdges)
   }
 
-  override def statistics: Map[String, Any] = {
-    // Get the total number of vertices after replication, used to compute the replication ratio.
-    def numReplicatedVertices(vid2pids: RDD[Array[Array[VertexID]]]): Double = {
-      vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble
-    }
-
-    val numVertices = this.ops.numVertices
-    val numEdges = this.ops.numEdges
-    val replicationRatioBoth = numReplicatedVertices(routingTable.bothAttrs) / numVertices
-    val replicationRatioSrcOnly = numReplicatedVertices(routingTable.srcAttrOnly) / numVertices
-    val replicationRatioDstOnly = numReplicatedVertices(routingTable.dstAttrOnly) / numVertices
-    // One entry for each partition, indicate the total number of edges on that partition.
-    val loadArray = edges.partitionsRDD.map(_._2.size).collect().map(_.toDouble / numEdges)
-    val minLoad = loadArray.min
-    val maxLoad = loadArray.max
-    Map(
-      "Num Vertices" -> numVertices,
-      "Num Edges" -> numEdges,
-      "Replication (both)" -> replicationRatioBoth,
-      "Replication (src only)" -> replicationRatioSrcOnly,
-      "Replication (dest only)" -> replicationRatioDstOnly,
-      "Load Array" -> loadArray,
-      "Min Load" -> minLoad,
-      "Max Load" -> maxLoad)
-  }
-
-  /**
-   * Display the lineage information for this graph.
-   */
-  def printLineage() = {
-    def traverseLineage(
-        rdd: RDD[_],
-        indent: String = "",
-        visited: Map[Int, String] = Map.empty[Int, String]) {
-      if (visited.contains(rdd.id)) {
-        println(indent + visited(rdd.id))
-        println(indent)
-      } else {
-        val locs = rdd.partitions.map( p => rdd.preferredLocations(p) )
-        val cacheLevel = rdd.getStorageLevel
-        val name = rdd.id
-        val deps = rdd.dependencies
-        val partitioner = rdd.partitioner
-        val numparts = partitioner match { case Some(p) => p.numPartitions; case None => 0}
-        println(indent + name + ": " + cacheLevel.description + " (partitioner: " + partitioner +
-          ", " + numparts +")")
-        println(indent + " |--->  Deps:    " + deps.map(d => (d, d.rdd.id) ).toString)
-        println(indent + " |--->  PrefLoc: " + locs.map(x=> x.toString).mkString(", "))
-        deps.foreach(d => traverseLineage(d.rdd, indent + " | ", visited))
-      }
-    }
-    println("edges ------------------------------------------")
-    traverseLineage(edges, "  ")
-    var visited = Map(edges.id -> "edges")
-    println("\n\nvertices ------------------------------------------")
-    traverseLineage(vertices, "  ", visited)
-    visited += (vertices.id -> "vertices")
-    println("\n\nroutingTable.bothAttrs -------------------------------")
-    traverseLineage(routingTable.bothAttrs, "  ", visited)
-    visited += (routingTable.bothAttrs.id -> "routingTable.bothAttrs")
-    println("\n\ntriplets ----------------------------------------")
-    traverseLineage(triplets, "  ", visited)
-    println(visited)
-  } // end of printLineage
-
   override def reverse: Graph[VD, ED] = {
     val newETable = edges.mapEdgePartitions((pid, part) => part.reverse)
     new GraphImpl(vertices, newETable, routingTable, replicatedVertexView)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d4d9ece1/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
index 08256dc..2f4d6d6 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
@@ -73,9 +73,6 @@ object PageRank extends Logging {
       .mapVertices( (id, attr) => 1.0 )
       .cache()
 
-    // Display statistics about pagerank
-    logInfo(pagerankGraph.statistics.toString)
-
     // Define the three functions needed to implement PageRank in the GraphX
     // version of Pregel
     def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double =
@@ -121,9 +118,6 @@ object PageRank extends Logging {
       .mapVertices( (id, attr) => (0.0, 0.0) )
       .cache()
 
-    // Display statistics about pagerank
-    logInfo(pagerankGraph.statistics.toString)
-
     // Define the three functions needed to implement PageRank in the GraphX
     // version of Pregel
     def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = {


[45/50] git commit: Fix bug in GraphLoader.edgeListFile that caused srcId > dstId

Posted by pw...@apache.org.
Fix bug in GraphLoader.edgeListFile that caused srcId > dstId


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/76ebdae7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/76ebdae7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/76ebdae7

Branch: refs/heads/master
Commit: 76ebdae79866f4721ef39aaccfa89a255d3cea6a
Parents: c6dbfd1
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 22:20:45 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 22:20:45 2014 -0800

----------------------------------------------------------------------
 graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/76ebdae7/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
index 7bdb101..5904aa3 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
@@ -53,7 +53,7 @@ object GraphLoader extends Logging {
           }
           val srcId = lineArray(0).toLong
           val dstId = lineArray(1).toLong
-          if (canonicalOrientation && dstId > srcId) {
+          if (canonicalOrientation && srcId > dstId) {
             builder.add(dstId, srcId, 1)
           } else {
             builder.add(srcId, dstId, 1)


[07/50] git commit: adding Pregel as an operator in GraphOps and cleaning up documentation of GraphOps

Posted by pw...@apache.org.
adding Pregel as an operator in GraphOps and cleaning up documentation of GraphOps


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/2216319f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/2216319f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/2216319f

Branch: refs/heads/master
Commit: 2216319f485ca2d00a946c4478dedc8a0e1c6053
Parents: c787ff5
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Sun Jan 12 21:26:37 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Sun Jan 12 21:26:37 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/graphx/GraphOps.scala      | 92 +++++++++++++++-----
 .../scala/org/apache/spark/graphx/Pregel.scala  |  4 +-
 2 files changed, 74 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2216319f/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 0121cb1..4fdff29 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -6,10 +6,9 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.SparkContext._
 import org.apache.spark.SparkException
 
-
 /**
  * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the
- * efficient GraphX API. This class  is implicitly constructed for each Graph object.
+ * efficient GraphX API. This class is implicitly constructed for each Graph object.
  *
  * @tparam VD the vertex attribute type
  * @tparam ED the edge attribute type
@@ -19,32 +18,27 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
   /** The number of edges in the graph. */
   lazy val numEdges: Long = graph.edges.count()
 
-
   /** The number of vertices in the graph. */
   lazy val numVertices: Long = graph.vertices.count()
 
-
   /**
    * The in-degree of each vertex in the graph.
    * @note Vertices with no in-edges are not returned in the resulting RDD.
    */
   lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In)
 
-
   /**
    * The out-degree of each vertex in the graph.
    * @note Vertices with no out-edges are not returned in the resulting RDD.
    */
   lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out)
 
-
   /**
    * The degree of each vertex in the graph.
    * @note Vertices with no edges are not returned in the resulting RDD.
    */
   lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Both)
 
-
   /**
    * Computes the neighboring vertex degrees.
    *
@@ -76,10 +70,10 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    * age for each user:
    *
    * {{{
-   * val graph: Graph[Int,Int] = loadGraph()
+   * val graph: Graph[Int,Int] = GraphLoader.edgeListFile(sc, "webgraph")
    * val averageFollowerAge: RDD[(Int, Int)] =
    *   graph.aggregateNeighbors[(Int,Double)](
-   *     (vid, edge) => (edge.otherVertex(vid).data, 1),
+   *     (vid, edge) => Some((edge.otherVertex(vid).data, 1)),
    *     (a, b) => (a._1 + b._1, a._2 + b._2),
    *     -1,
    *     EdgeDirection.In)
@@ -111,11 +105,9 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
         case (Some(srcA), Some(dstA)) => Iterator((et.srcId, srcA), (et.dstId, dstA))
       }
     }
-
     graph.mapReduceTriplets(mf, reduceFunc)
   } // end of aggregateNeighbors
 
-
   /**
    * Collect the neighbor vertex ids for each vertex.
    *
@@ -147,7 +139,6 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
     }
   } // end of collectNeighborIds
 
-
   /**
    * Collect the neighbor vertex attributes for each vertex.
    *
@@ -173,7 +164,6 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
     }
   } // end of collectNeighbor
 
-
   /**
    * Join the vertices with an RDD and then apply a function from the
    * the vertex and RDD entry to a new vertex value.  The input table
@@ -188,17 +178,14 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    * corresponding entry in the table otherwise the old vertex value
    * is used.
    *
-   * @note for small tables this function can be much more efficient
-   * than leftJoinVertices
-   *
    * @example This function is used to update the vertices with new
    * values based on external data.  For example we could add the out
    * degree to each vertex record
    *
    * {{{
-   * val rawGraph: Graph[Int,()] = Graph.textFile("webgraph")
+   * val rawGraph: Graph[Int, Int] = GraphLoader.edgeListFile(sc, "webgraph")
    *   .mapVertices(v => 0)
-   * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees()
+   * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees
    * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg,
    *   (v, deg) => deg )
    * }}}
@@ -219,8 +206,10 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    * Filter the graph by computing some values to filter on, and applying the predicates.
    *
    * @param preprocess a function to compute new vertex and edge data before filtering
-   * @param epred edge pred to filter on after preprocess, see more details under Graph#subgraph
-   * @param vpred vertex pred to filter on after prerocess, see more details under Graph#subgraph
+   * @param epred edge pred to filter on after preprocess, see more details under
+   *  [[org.apache.spark.graphx.Graph#subgraph]]
+   * @param vpred vertex pred to filter on after prerocess, see more details under
+   *  [[org.apache.spark.graphx.Graph#subgraph]]
    * @tparam VD2 vertex type the vpred operates on
    * @tparam ED2 edge type the epred operates on
    * @return a subgraph of the orginal graph, with its data unchanged
@@ -246,4 +235,67 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
       vpred: (VertexID, VD2) => Boolean = (v:VertexID, d:VD2) => true): Graph[VD, ED] = {
     graph.mask(preprocess(graph).subgraph(epred, vpred))
   }
+
+  /**
+   * Execute a Pregel-like iterative vertex-parallel abstraction.  The
+   * user-defined vertex-program `vprog` is executed in parallel on
+   * each vertex receiving any inbound messages and computing a new
+   * value for the vertex.  The `sendMsg` function is then invoked on
+   * all out-edges and is used to compute an optional message to the
+   * destination vertex. The `mergeMsg` function is a commutative
+   * associative function used to combine messages destined to the
+   * same vertex.
+   *
+   * On the first iteration all vertices receive the `initialMsg` and
+   * on subsequent iterations if a vertex does not receive a message
+   * then the vertex-program is not invoked.
+   *
+   * This function iterates until there are no remaining messages, or
+   * for `maxIterations` iterations.
+   *
+   * @tparam VD the vertex data type
+   * @tparam ED the edge data type
+   * @tparam A the Pregel message type
+   *
+   * @param graph the input graph.
+   *
+   * @param initialMsg the message each vertex will receive at the on
+   * the first iteration
+   *
+   * @param maxIterations the maximum number of iterations to run for
+   *
+   * @param activeDirection the direction of edges incident to a vertex that received a message in
+   * the previous round on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only
+   * out-edges of vertices that received a message in the previous round will run.
+   *
+   * @param vprog the user-defined vertex program which runs on each
+   * vertex and receives the inbound message and computes a new vertex
+   * value.  On the first iteration the vertex program is invoked on
+   * all vertices and is passed the default message.  On subsequent
+   * iterations the vertex program is only invoked on those vertices
+   * that receive messages.
+   *
+   * @param sendMsg a user supplied function that is applied to out
+   * edges of vertices that received messages in the current
+   * iteration
+   *
+   * @param mergeMsg a user supplied function that takes two incoming
+   * messages of type A and merges them into a single message of type
+   * A.  ''This function must be commutative and associative and
+   * ideally the size of A should not increase.''
+   *
+   * @return the resulting graph at the end of the computation
+   *
+   */
+  def pregel[A: ClassTag](
+      initialMsg: A,
+      maxIterations: Int = Int.MaxValue,
+      activeDirection: EdgeDirection = EdgeDirection.Out)(
+      vprog: (VertexID, VD, A) => VD,
+      sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
+      mergeMsg: (A, A) => A)
+    : Graph[VD, ED] = {
+    Pregel(graph, initialMsg, maxIterations, activeDirection)(vprog, sendMsg, mergeMsg)
+  }
+
 } // end of GraphOps

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/2216319f/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
index 57b0872..83e28d0 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
@@ -25,8 +25,8 @@ import scala.reflect.ClassTag
  *
  * def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double =
  *   resetProb + (1.0 - resetProb) * msgSum
- * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Option[Double] =
- *   Some(edge.srcAttr * edge.attr)
+ * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Iterator[(VertexId, Double)] =
+ *   Iterator((edge.dstId, edge.srcAttr * edge.attr))
  * def messageCombiner(a: Double, b: Double): Double = a + b
  * val initialMessage = 0.0
  * // Execute Pregel for a fixed number of iterations.


[13/50] git commit: Further improve VertexRDD scaladocs

Posted by pw...@apache.org.
Further improve VertexRDD scaladocs


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ea69cff7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ea69cff7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ea69cff7

Branch: refs/heads/master
Commit: ea69cff7110e4eaf4e92bc3f2f9d1e765f172b0e
Parents: 8ca9773
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 12:52:52 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 12:52:52 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/graphx/VertexRDD.scala     | 39 +++++++++++++-------
 1 file changed, 25 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ea69cff7/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
index d20745d..9a95364 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
@@ -28,17 +28,14 @@ import org.apache.spark.graphx.impl.MsgRDDFunctions
 import org.apache.spark.graphx.impl.VertexPartition
 
 /**
- * `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is only one entry for
- * each vertex and by pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the
- * same index can be joined efficiently.
+ * Extends `RDD[(VertexID, VD)]` by ensuring that there is only one entry for each vertex and by
+ * pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the same index can be
+ * joined efficiently. All operations except [[reindex]] preserve the index. To construct a
+ * `VertexRDD`, use the [[org.apache.spark.graphx.VertexRDD$ VertexRDD object]].
  *
- * @tparam VD the vertex attribute associated with each vertex in the set.
- *
- * To construct a `VertexRDD` use the singleton object:
- *
- * @example Construct a `VertexRDD` from a plain RDD
+ * @example Construct a `VertexRDD` from a plain RDD:
  * {{{
- * // Construct an intial vertex set
+ * // Construct an initial vertex set
  * val someData: RDD[(VertexID, SomeType)] = loadData(someFile)
  * val vset = VertexRDD(someData)
  * // If there were redundant values in someData we would use a reduceFunc
@@ -50,6 +47,7 @@ import org.apache.spark.graphx.impl.VertexPartition
  * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3)
  * }}}
  *
+ * @tparam VD the vertex attribute associated with each vertex in the set.
  */
 class VertexRDD[@specialized VD: ClassTag](
     val partitionsRDD: RDD[VertexPartition[VD]])
@@ -146,7 +144,7 @@ class VertexRDD[@specialized VD: ClassTag](
     this.mapVertexPartitions(_.map(f))
 
   /**
-   * Hides vertices that are the same between `this` and `other`. For vertices that are different,
+   * Hides vertices that are the same between `this` and `other`; for vertices that are different,
    * keeps the values from `other`.
    */
   def diff(other: VertexRDD[VD]): VertexRDD[VD] = {
@@ -188,7 +186,7 @@ class VertexRDD[@specialized VD: ClassTag](
   /**
    * Left joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is
    * backed by a VertexRDD with the same index then the efficient [[leftZipJoin]] implementation is
-   * used. The resulting vertex set contains an entry for each vertex in this set. If `other` is
+   * used. The resulting VertexRDD contains an entry for each vertex in `this`. If `other` is
    * missing any vertex in this VertexRDD, `f` is passed `None`. If there are duplicates, the vertex
    * is picked arbitrarily.
    *
@@ -223,8 +221,8 @@ class VertexRDD[@specialized VD: ClassTag](
   }
 
   /**
-   * Same effect as `leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }`, but `this` and
-   * `other` must have the same index.
+   * Efficiently inner joins this VertexRDD with another VertexRDD sharing the same index. See
+   * [[innerJoin]] for the behavior of the join.
    */
   def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U])
       (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = {
@@ -242,6 +240,12 @@ class VertexRDD[@specialized VD: ClassTag](
    * Inner joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is
    * backed by a VertexRDD with the same index then the efficient [[innerZipJoin]] implementation is
    * used.
+   *
+   * @param other an RDD containing vertices to join. If there are multiple entries for the same
+   * vertex, one is picked arbitrarily. Use [[aggregateUsingIndex]] to merge multiple entries.
+   * @param f the join function applied to corresponding values of `this` and `other`
+   * @return a VertexRDD co-indexed with `this`, containing only vertices that appear in both `this`
+   * and `other`, with values supplied by `f`
    */
   def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)])
       (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = {
@@ -263,8 +267,15 @@ class VertexRDD[@specialized VD: ClassTag](
   }
 
   /**
-   * Aggregates vertices in `message` that have the same ids using `reduceFunc`, returning a
+   * Aggregates vertices in `messages` that have the same ids using `reduceFunc`, returning a
    * VertexRDD co-indexed with `this`.
+   *
+   * @param messages an RDD containing messages to aggregate, where each message is a pair of its
+   * target vertex ID and the message data
+   * @param reduceFunc the associative aggregation function for merging messages to the same vertex
+   * @return a VertexRDD co-indexed with `this`, containing only vertices that received messages.
+   * For those vertices, their values are the result of applying `reduceFunc` to all received
+   * messages.
    */
   def aggregateUsingIndex[VD2: ClassTag](
       messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = {


[30/50] git commit: Finished second pass on pregel docs.

Posted by pw...@apache.org.
Finished second pass on pregel docs.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/552de5d4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/552de5d4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/552de5d4

Branch: refs/heads/master
Commit: 552de5d42e395bad19f5d5fe6dcc1e678bb994a8
Parents: 622b7f7
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Mon Jan 13 18:40:35 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Mon Jan 13 18:40:43 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 45 +++++++++++++++++++++++++----------
 1 file changed, 33 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/552de5d4/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index c6505d2..77d8078 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -484,10 +484,28 @@ messages destined to each vertex.  The `mapReduceTriplets` operator returns a `V
 containing the aggregate message (of type `A`) destined to each vertex.  Vertices that do not
 receive a message are not included in the returned `VertexRDD`.
 
-> Note that `mapReduceTriplets` takes an additional optional `activeSet` (see API docs) which
-> restricts the map phase to edges adjacent to the vertices in the provided `VertexRDD`. Restricting
-> computation to triplets adjacent to a subset of the vertices is often necessary in incremental
-> iterative computation and is a key part of the GraphX implementation of Pregel.
+<blockquote>
+<p>
+Note that <code>mapReduceTriplets</code> takes an additional optional <code>activeSet</code>
+(see API docs) which restricts the map phase to edges adjacent to the vertices in the provided
+<code>VertexRDD</code>:
+</p>
+{% highlight scala %}
+  activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None
+{% endhighlight %}
+<p>
+The EdgeDirection specifies which edges adjacent to the vertex set are included in the map phase. If
+the direction is <code>In</code>, <code>mapFunc</code> will only be run only on edges with
+destination in the active set. If the direction is <code>Out</code>, <code>mapFunc</code> will only
+be run only on edges originating from vertices in the active set.  If the direction is
+<code>Either</code>, <code>mapFunc</code> will be run only on edges with <i>either</i> vertex in the
+active set.  If the direction is <code>Both</code>, <code>mapFunc</code> will be run only on edges
+with both vertices in the active set.  The active set must be derived from the set of vertices in
+the graph. Restricting computation to triplets adjacent to a subset of the vertices is often
+necessary in incremental iterative computation and is a key part of the GraphX implementation of
+Pregel.
+</p>
+</blockquote>
 
 In the following example we use the `mapReduceTriplets` operator to compute the average age of the
 more senior followers of each user.
@@ -565,15 +583,18 @@ Graphs are inherently recursive data-structures as properties of vertices depend
 their neighbors which intern depend on properties of *their* neighbors.  As a
 consequence many important graph algorithms iteratively recompute the properties of each vertex
 until a fixed-point condition is reached.  A range of graph-parallel abstractions have been proposed
-to express these iterative algorithms.  GraphX exposes a Pregel operator which is a fusion of
+to express these iterative algorithms.  GraphX exposes a Pregel-like operator which is a fusion of
 the widely used Pregel and GraphLab abstractions.
 
-At a high-level the GraphX variant of the Pregel abstraction is a bulk-synchronous parallel
-messaging abstraction constrained to the topology of the graph.  The Pregel operator executes in a
-series of super-steps in which vertices receive the sum of their inbound messages from the previous
-super-step, compute a new property value, and then send messages to neighboring vertices in the next
-super-step.  Vertices that do not receive a message are skipped within a super-step.  The Pregel
-operators terminates iteration and returns the final graph when there are no messages remaining.
+At a high-level the Pregel operator in GraphX is a bulk-synchronous parallel messaging abstraction
+*constrained to the topology of the graph*.  The Pregel operator executes in a series of super-steps
+in which vertices receive the *sum* of their inbound messages from the previous super- step, compute
+a new value for the vertex property, and then send messages to neighboring vertices in the next
+super-step.  Unlike Pregel and instead more like GraphLab messages are computed in parallel as a
+function of the edge triplet and the message computation has access to both the source and
+destination vertex attributes.  Vertices that do not receive a message are skipped within a super-
+step.  The Pregel operators terminates iteration and returns the final graph when there are no
+messages remaining.
 
 > Note, unlike more standard Pregel implementations, vertices in GraphX can only send messages to
 > neighboring vertices and the message construction is done in parallel using a user defined
@@ -588,7 +609,7 @@ def pregel[A]
      maxIter: Int = Int.MaxValue,
      activeDir: EdgeDirection = EdgeDirection.Out)
     (vprog: (VertexID, VD, A) => VD,
-     sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
+     sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
      mergeMsg: (A, A) => A)
   : Graph[VD, ED] = {
   // Receive the initial message at each vertex


[27/50] git commit: Updated GraphGenerator.

Posted by pw...@apache.org.
Updated GraphGenerator.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ae06d2c2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ae06d2c2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ae06d2c2

Branch: refs/heads/master
Commit: ae06d2c22ffb8af3c27c29bc55aadfb73b56e9ff
Parents: 87f335d
Author: Reynold Xin <rx...@apache.org>
Authored: Mon Jan 13 18:31:49 2014 -0800
Committer: Reynold Xin <rx...@apache.org>
Committed: Mon Jan 13 18:31:49 2014 -0800

----------------------------------------------------------------------
 .../spark/graphx/util/GraphGenerators.scala     | 60 ++++++++++----------
 1 file changed, 30 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ae06d2c2/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
index e0fd9b9..dbea233 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
@@ -15,6 +15,7 @@ import org.apache.spark.graphx.Graph
 import org.apache.spark.graphx.Edge
 import org.apache.spark.graphx.impl.GraphImpl
 
+/** A collection of graph generating functions. */
 object GraphGenerators {
 
   val RMATa = 0.45
@@ -24,6 +25,9 @@ object GraphGenerators {
 
   // Right now it just generates a bunch of edges where
   // the edge data is the weight (default 1)
+  /**
+   * Generate a graph whose vertex out degree is log normal.
+   */
   def logNormalGraph(sc: SparkContext, numVertices: Int): Graph[Int, Int] = {
     // based on Pregel settings
     val mu = 4
@@ -32,13 +36,12 @@ object GraphGenerators {
     val vertices: RDD[(VertexID, Int)] = sc.parallelize(0 until numVertices).map{
       src => (src, sampleLogNormal(mu, sigma, numVertices))
     }
-    val edges = vertices.flatMap{
-      v => generateRandomEdges(v._1.toInt, v._2, numVertices)
+    val edges = vertices.flatMap { v =>
+      generateRandomEdges(v._1.toInt, v._2, numVertices)
     }
     Graph(vertices, edges, 0)
   }
 
-
   def generateRandomEdges(src: Int, numEdges: Int, maxVertexID: Int): Array[Edge[Int]] = {
     val rand = new Random()
     var dsts: Set[Int] = Set()
@@ -48,10 +51,9 @@ object GraphGenerators {
         dsts += nextDst
       }
     }
-    dsts.map {dst => Edge[Int](src, dst, 1) }.toArray
+    dsts.map(dst => Edge[Int](src, dst, 1)).toArray
   }
 
-
   /**
    * Randomly samples from a log normal distribution whose corresponding normal distribution has the
    * the given mean and standard deviation. It uses the formula `X = exp(m+s*Z)` where `m`, `s` are
@@ -60,9 +62,9 @@ object GraphGenerators {
    *
    * @param mu the mean of the normal distribution
    * @param sigma the standard deviation of the normal distribution
-   * @param macVal exclusive upper bound on the value of the sample
+   * @param maxVal exclusive upper bound on the value of the sample
    */
-  def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = {
+  private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = {
     val rand = new Random()
     val m = math.exp(mu+(sigma*sigma)/2.0)
     val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma))
@@ -76,27 +78,29 @@ object GraphGenerators {
     math.round(X.toFloat)
   }
 
-
-
+  /**
+   * A random graph generator using the R-MAT model, proposed in
+   * "R-MAT: A Recursive Model for Graph Mining" by Chakrabarti et al.
+   *
+   * See [[http://www.cs.cmu.edu/~christos/PUBLICATIONS/siam04.pdf]].
+   */
   def rmatGraph(sc: SparkContext, requestedNumVertices: Int, numEdges: Int): Graph[Int, Int] = {
     // let N = requestedNumVertices
     // the number of vertices is 2^n where n=ceil(log2[N])
     // This ensures that the 4 quadrants are the same size at all recursion levels
-    val numVertices = math.round(math.pow(2.0, math.ceil(math.log(requestedNumVertices)/math.log(2.0)))).toInt
+    val numVertices = math.round(
+      math.pow(2.0, math.ceil(math.log(requestedNumVertices) / math.log(2.0)))).toInt
     var edges: Set[Edge[Int]] = Set()
     while (edges.size < numEdges) {
       if (edges.size % 100 == 0) {
         println(edges.size + " edges")
       }
       edges += addEdge(numVertices)
-
     }
-    val graph = outDegreeFromEdges(sc.parallelize(edges.toList))
-    graph
-
+    outDegreeFromEdges(sc.parallelize(edges.toList))
   }
 
-  def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = {
+  private def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = {
     val vertices = edges.flatMap { edge => List((edge.srcId, 1)) }
       .reduceByKey(_ + _)
       .map{ case (vid, degree) => (vid, degree) }
@@ -107,7 +111,7 @@ object GraphGenerators {
    * @param numVertices Specifies the total number of vertices in the graph (used to get
    * the dimensions of the adjacency matrix
    */
-  def addEdge(numVertices: Int): Edge[Int] = {
+  private def addEdge(numVertices: Int): Edge[Int] = {
     //val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0)
     val v = math.round(numVertices.toFloat/2.0).toInt
 
@@ -115,7 +119,6 @@ object GraphGenerators {
     Edge[Int](src, dst, 1)
   }
 
-
   /**
    * This method recursively subdivides the the adjacency matrix into quadrants
    * until it picks a single cell. The naming conventions in this paper match
@@ -149,10 +152,10 @@ object GraphGenerators {
    * }}}
    */
   @tailrec
-  def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = {
-    if (t <= 1)
-      (x,y)
-    else {
+  private def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = {
+    if (t <= 1) {
+      (x, y)
+    } else {
       val newT = math.round(t.toFloat/2.0).toInt
       pickQuadrant(RMATa, RMATb, RMATc, RMATd) match {
         case 0 => chooseCell(x, y, newT)
@@ -164,22 +167,21 @@ object GraphGenerators {
   }
 
   // TODO(crankshaw) turn result into an enum (or case class for pattern matching}
-  def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = {
-    if (a+b+c+d != 1.0) {
-      throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a+b+c+d) + ", should sum to 1.0")
+  private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = {
+    if (a + b + c + d != 1.0) {
+      throw new IllegalArgumentException(
+        "R-MAT probability parameters sum to " + (a+b+c+d) + ", should sum to 1.0")
     }
     val rand = new Random()
     val result = rand.nextDouble()
     result match {
       case x if x < a => 0 // 0 corresponds to quadrant a
-      case x if (x >= a && x < a+b) => 1 // 1 corresponds to b
-      case x if (x >= a+b && x < a+b+c) => 2 // 2 corresponds to c
+      case x if (x >= a && x < a + b) => 1 // 1 corresponds to b
+      case x if (x >= a + b && x < a + b + c) => 2 // 2 corresponds to c
       case _ => 3 // 3 corresponds to d
     }
   }
 
-
-
   /**
    * Create `rows` by `cols` grid graph with each vertex connected to its
    * row+1 and col+1 neighbors.  Vertex ids are assigned in row major
@@ -220,6 +222,4 @@ object GraphGenerators {
     Graph.fromEdgeTuples(edges, 1)
   } // end of starGraph
 
-
-
 } // end of Graph Generators


[35/50] git commit: Finished documenting vertexrdd.

Posted by pw...@apache.org.
Finished documenting vertexrdd.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ee8931d2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ee8931d2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ee8931d2

Branch: refs/heads/master
Commit: ee8931d2c6503716de640d6d1249c515e1fd85d3
Parents: 0fbc0b0
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Mon Jan 13 19:30:25 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Mon Jan 13 19:30:35 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 53 +++++++++++++++++++++++++++++++++++
 1 file changed, 53 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ee8931d2/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 77d8078..76de26c 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -683,7 +683,60 @@ val sssp = initialGraph.pregel(Double.PositiveInfinity)(
 # Vertex and Edge RDDs
 <a name="vertex_and_edge_rdds"></a>
 
+GraphX exposes `RDD` views of the vertices and edges stored within the graph.  However, because
+GraphX maintains the vertices and edges in optimized data-structures and these data-structures
+provide additional functionality, the vertices and edges are returned as `VertexRDD` and `EdgeRDD`
+respectively.  In this section we review some of the additional useful functionality in these types.
 
+## VertexRDDs
+
+The `VertexRDD[A]` extends the more traditional `RDD[(VertexId, A)]` but adds the additional
+constraint that each `VertexId` occurs only *once*.  Moreover, `VertexRDD[A]` represents a *set* of
+vertices each with an attribute of type `A`.  Internally, this is achieved by storing the vertex
+attributes in a reusable hash-map data-structure.  As a consequence if two `VertexRDD`s are derived
+from the same base `VertexRDD` (e.g., by `filter` or `mapValues`) they can be joined in constant
+time without hash evaluations. To leverage this indexed data-structure, the `VertexRDD` exposes the
+following additional functionality:
+
+{% highlight scala %}
+// Filter the vertex set but preserves the internal index
+def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD]
+// Transform the values without changing the ids (preserves the internal index)
+def mapValues[VD2](map: VD => VD2): VertexRDD[VD2]
+def mapValues[VD2](map: (VertexID, VD) => VD2): VertexRDD[VD2]
+// Remove vertices from this set that appear in the other set
+def diff(other: VertexRDD[VD]): VertexRDD[VD]
+// Join operators that take advantage of the internal indexing to accelerate joins (substantially)
+def leftJoin[VD2, VD3](other: RDD[(VertexID, VD2)])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3]
+def innerJoin[U, VD2](other: RDD[(VertexID, U)])(f: (VertexID, VD, U) => VD2): VertexRDD[VD2]
+// Use the index on this RDD to accelerate a `reduceByKey` operation on the input RDD.
+def aggregateUsingIndex[VD2](other: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2]
+{% endhighlight %}
+
+Notice, for example,  how the `filter` operator returns an `VertexRDD`.  Filter is actually
+implemented using a `BitSet` thereby reusing the index and preserving the ability to do fast joins
+with other `VertexRDD`s.  Likewise, the `mapValues` operators do not allow the `map` function to
+change the `VertexId` thereby enabling the same `HashMap` data-structures to be reused.  Both the
+`leftJoin` and `innerJoin` are able to identify when joining two `VertexRDD`s derived from the same
+`HashMap` and implement the join by linear scan rather than costly point lookups.
+
+The `aggregateUsingIndex` operator can be slightly confusing but is also useful for efficient
+construction of a new `VertexRDD` from an `RDD[(VertexId, A)]`.  Conceptually, if I have constructed
+a `VertexRDD[B]` over a set of vertices, *which is a super-set* of the vertices in some
+`RDD[(VertexId, A)]` then I can reuse the index to both aggregate and then subsequently index the
+RDD.  For example:
+
+{% highlight scala %}
+val setA: VertexRDD[Int] = VertexRDD(sc.parallelize(0L until 100L).map(id => (id, 1)))
+val rddB: RDD[(VertexID, Double)] = sc.parallelize(0L until 100L).flatMap(id => List((id, 1.0), (id, 2.0)))
+// There should be 200 entries in rddB
+rddB.count
+val setB: VertexRDD[Double] = setA.aggregateUsingIndex(rddB, _ + _)
+// There should be 100 entries in setB
+setB.count
+// Joining A and B should now be fast!
+val setC: VertexRDD[Double] = setA.innerJoin(setB)((id, a, b) => a + b)
+{% endhighlight %}
 
 # Optimized Representation
 


[17/50] git commit: Add graph loader links to doc

Posted by pw...@apache.org.
Add graph loader links to doc


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/97cd27e3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/97cd27e3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/97cd27e3

Branch: refs/heads/master
Commit: 97cd27e31b18f4c41ef556aee2ab65350694f8b8
Parents: 15ca89b
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 14:54:48 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 14:54:48 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97cd27e3/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 1832ded..7f1559d 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -638,6 +638,19 @@ val sssp = initialGraph.pregel(Double.PositiveInfinity)(
 # Graph Builders
 <a name="graph_builders"></a>
 
+[`GraphLoader.edgeListFile`][GraphLoader.edgeListFile]
+
+[`Graph.apply`][Graph.apply]
+
+[`Graph.fromEdgeTuples`][Graph.fromEdgeTuples]
+
+[`Graph.fromEdges`][Graph.fromEdges]
+
+[GraphLoader.edgeListFile]: api/graphx/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int]
+[Graph.apply]: api/graphx/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexID,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED]
+[Graph.fromEdgeTuples]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexID,VertexID)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int]
+[Graph.fromEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED]
+
 # Vertex and Edge RDDs
 <a name="vertex_and_edge_rdds"></a>
 


[16/50] git commit: Fix mapReduceTriplets links in doc

Posted by pw...@apache.org.
Fix mapReduceTriplets links in doc


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/15ca89b1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/15ca89b1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/15ca89b1

Branch: refs/heads/master
Commit: 15ca89b11edbb2800efc992d6cf4eba787a00873
Parents: 9fe8862
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 14:54:33 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 14:54:33 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/15ca89b1/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 2697b2d..1832ded 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -443,10 +443,10 @@ PageRank Value, shortest path to the source, and smallest reachable vertex id).
 ### Map Reduce Triplets (mapReduceTriplets)
 <a name="mrTriplets"></a>
 
-[Graph.mapReduceTriplets]: api/graphx/index.html#mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=&gt;Iterator[(org.apache.spark.graphx.VertexID,A)],reduceFunc:(A,A)=&gt;A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A]
+[Graph.mapReduceTriplets]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=&gt;Iterator[(org.apache.spark.graphx.VertexID,A)],reduceFunc:(A,A)=&gt;A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A]
 
-These core (heavily optimized) aggregation primitive in GraphX is the
-(`mapReduceTriplets`)[Graph.mapReduceTriplets] operator:
+The core (heavily optimized) aggregation primitive in GraphX is the
+[`mapReduceTriplets`][Graph.mapReduceTriplets] operator:
 
 {% highlight scala %}
 def mapReduceTriplets[A](
@@ -455,7 +455,7 @@ def mapReduceTriplets[A](
   : VertexRDD[A]
 {% endhighlight %}
 
-The (`mapReduceTriplets`)[Graph.mapReduceTriplets] operator takes a user defined map function which
+The [`mapReduceTriplets`][Graph.mapReduceTriplets] operator takes a user defined map function which
 is applied to each triplet and can yield *messages* destined to either (none or both) vertices in
 the triplet.  We currently only support messages destined to the source or destination vertex of the
 triplet to enable optimized preaggregation.  The user defined `reduce` function combines the


[43/50] git commit: Wrap methods in the appropriate class/object declaration

Posted by pw...@apache.org.
Wrap methods in the appropriate class/object declaration


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/6f6f8c92
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/6f6f8c92
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/6f6f8c92

Branch: refs/heads/master
Commit: 6f6f8c928ce493357d4d32e46971c5e401682ea8
Parents: 67795db
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 21:55:35 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:55:35 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 149 +++++++++++++++++++---------------
 1 file changed, 85 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6f6f8c92/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index aadeb38..29d397c 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -256,7 +256,7 @@ compute the in-degree of each vertex (defined in `GraphOps`) by the following:
 {% highlight scala %}
 val graph: Graph[(String, String), String]
 // Use the implicit GraphOps.inDegrees operator
-val indDegrees: VertexRDD[Int] = graph.inDegrees
+val inDegrees: VertexRDD[Int] = graph.inDegrees
 {% endhighlight %}
 
 The reason for differentiating between core graph operations and [`GraphOps`][GraphOps] is to be
@@ -270,9 +270,11 @@ In direct analogy to the RDD `map` operator, the property
 graph contains the following:
 
 {% highlight scala %}
-def mapVertices[VD2](map: (VertexID, VD) => VD2): Graph[VD2, ED]
-def mapEdges[ED2](map: Edge[ED] => ED2): Graph[VD, ED2]
-def mapTriplets[ED2](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2]
+class Graph[VD, ED] {
+  def mapVertices[VD2](map: (VertexID, VD) => VD2): Graph[VD2, ED]
+  def mapEdges[ED2](map: Edge[ED] => ED2): Graph[VD, ED2]
+  def mapTriplets[ED2](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2]
+}
 {% endhighlight %}
 
 Each of these operators yields a new graph with the vertex or edge properties modified by the user
@@ -314,11 +316,13 @@ Currently GraphX supports only a simple set of commonly used structural operator
 add more in the future.  The following is a list of the basic structural operators.
 
 {% highlight scala %}
-def reverse: Graph[VD, ED]
-def subgraph(epred: EdgeTriplet[VD,ED] => Boolean,
-             vpred: (VertexID, VD) => Boolean): Graph[VD, ED]
-def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED]
-def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED]
+class Graph[VD, ED] {
+  def reverse: Graph[VD, ED]
+  def subgraph(epred: EdgeTriplet[VD,ED] => Boolean,
+               vpred: (VertexID, VD) => Boolean): Graph[VD, ED]
+  def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED]
+  def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED]
+}
 {% endhighlight %}
 
 The [`reverse`][Graph.reverse] operator returns a new graph with all the edge directions reversed.
@@ -400,10 +404,12 @@ might want to pull vertex properties from one graph into another.  These tasks c
 using the *join* operators. Below we list the key join operators:
 
 {% highlight scala %}
-def joinVertices[U](table: RDD[(VertexID, U)])(map: (VertexID, VD, U) => VD)
-  : Graph[VD, ED]
-def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(map: (VertexID, VD, Option[U]) => VD2)
-  : Graph[VD2, ED]
+class Graph[VD, ED] {
+  def joinVertices[U](table: RDD[(VertexID, U)])(map: (VertexID, VD, U) => VD)
+    : Graph[VD, ED]
+  def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(map: (VertexID, VD, Option[U]) => VD2)
+    : Graph[VD2, ED]
+}
 {% endhighlight %}
 
 The [`joinVertices`][GraphOps.joinVertices] operator joins the vertices with the input RDD and
@@ -470,10 +476,12 @@ The core (heavily optimized) aggregation primitive in GraphX is the
 [`mapReduceTriplets`][Graph.mapReduceTriplets] operator:
 
 {% highlight scala %}
-def mapReduceTriplets[A](
-    map: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
-    reduce: (A, A) => A)
-  : VertexRDD[A]
+class Graph[VD, ED] {
+  def mapReduceTriplets[A](
+      map: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+      reduce: (A, A) => A)
+    : VertexRDD[A]
+}
 {% endhighlight %}
 
 The [`mapReduceTriplets`][Graph.mapReduceTriplets] operator takes a user defined map function which
@@ -564,12 +572,19 @@ val maxDegrees: (VertexID, Int)   = graph.degrees.reduce(max)
 ### Collecting Neighbors
 
 In some cases it may be easier to express computation by collecting neighboring vertices and their
-attributes at each vertex. This can be easily accomplished using the `collectNeighborIds` and the
-`collectNeighbors` operators.
+attributes at each vertex. This can be easily accomplished using the
+[`collectNeighborIds`][GraphOps.collectNeighborIds] and the
+[`collectNeighbors`][GraphOps.collectNeighbors] operators.
+
+[GraphOps.collectNeighborIds]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexID]]
+[GraphOps.collectNeighbors]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexID,VD)]]
+
 
 {% highlight scala %}
-def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]] =
-def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[ Array[(VertexID, VD)] ]
+class GraphOps[VD, ED] {
+  def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]]
+  def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[ Array[(VertexID, VD)] ]
+}
 {% endhighlight %}
 
 > Note that these operators can be quite costly as they duplicate information and require
@@ -600,40 +615,44 @@ messages remaining.
 > neighboring vertices and the message construction is done in parallel using a user defined
 > messaging function.  These constraints allow additional optimization within GraphX.
 
-The following is type signature of the Pregel operator as well as a *sketch* of its implementation
-(note calls to graph.cache have been removed):
+The following is the type signature of the [Pregel operator][GraphOps.pregel] as well as a *sketch*
+of its implementation (note calls to graph.cache have been removed):
+
+[GraphOps.pregel]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexID,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexID,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED]
 
 {% highlight scala %}
-def pregel[A]
-    (initialMsg: A,
-     maxIter: Int = Int.MaxValue,
-     activeDir: EdgeDirection = EdgeDirection.Out)
-    (vprog: (VertexID, VD, A) => VD,
-     sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
-     mergeMsg: (A, A) => A)
-  : Graph[VD, ED] = {
-  // Receive the initial message at each vertex
-  var g = mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache()
-  // compute the messages
-  var messages = g.mapReduceTriplets(sendMsg, mergeMsg)
-  var activeMessages = messages.count()
-  // Loop until no messages remain or maxIterations is achieved
-  var i = 0
-  while (activeMessages > 0 && i < maxIterations) {
-    // Receive the messages: -----------------------------------------------------------------------
-    // Run the vertex program on all vertices that receive messages
-    val newVerts = g.vertices.innerJoin(messages)(vprog).cache()
-    // Merge the new vertex values back into the graph
-    g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) }.cache()
-    // Send Messages: ------------------------------------------------------------------------------
-    // Vertices that didn't receive a message above don't appear in newVerts and therefore don't
-    // get to send messages.  More precisely the map phase of mapReduceTriplets is only invoked
-    // on edges in the activeDir of vertices in newVerts
-    messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, activeDir))).cache()
-    activeMessages = messages.count()
-    i += 1
+class GraphOps[VD, ED] {
+  def pregel[A]
+      (initialMsg: A,
+       maxIter: Int = Int.MaxValue,
+       activeDir: EdgeDirection = EdgeDirection.Out)
+      (vprog: (VertexID, VD, A) => VD,
+       sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+       mergeMsg: (A, A) => A)
+    : Graph[VD, ED] = {
+    // Receive the initial message at each vertex
+    var g = mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache()
+    // compute the messages
+    var messages = g.mapReduceTriplets(sendMsg, mergeMsg)
+    var activeMessages = messages.count()
+    // Loop until no messages remain or maxIterations is achieved
+    var i = 0
+    while (activeMessages > 0 && i < maxIterations) {
+      // Receive the messages: -----------------------------------------------------------------------
+      // Run the vertex program on all vertices that receive messages
+      val newVerts = g.vertices.innerJoin(messages)(vprog).cache()
+      // Merge the new vertex values back into the graph
+      g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) }.cache()
+      // Send Messages: ------------------------------------------------------------------------------
+      // Vertices that didn't receive a message above don't appear in newVerts and therefore don't
+      // get to send messages.  More precisely the map phase of mapReduceTriplets is only invoked
+      // on edges in the activeDir of vertices in newVerts
+      messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, activeDir))).cache()
+      activeMessages = messages.count()
+      i += 1
+    }
+    g
   }
-  g
 }
 {% endhighlight %}
 
@@ -749,18 +768,20 @@ time without hash evaluations. To leverage this indexed data-structure, the `Ver
 following additional functionality:
 
 {% highlight scala %}
-// Filter the vertex set but preserves the internal index
-def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD]
-// Transform the values without changing the ids (preserves the internal index)
-def mapValues[VD2](map: VD => VD2): VertexRDD[VD2]
-def mapValues[VD2](map: (VertexID, VD) => VD2): VertexRDD[VD2]
-// Remove vertices from this set that appear in the other set
-def diff(other: VertexRDD[VD]): VertexRDD[VD]
-// Join operators that take advantage of the internal indexing to accelerate joins (substantially)
-def leftJoin[VD2, VD3](other: RDD[(VertexID, VD2)])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3]
-def innerJoin[U, VD2](other: RDD[(VertexID, U)])(f: (VertexID, VD, U) => VD2): VertexRDD[VD2]
-// Use the index on this RDD to accelerate a `reduceByKey` operation on the input RDD.
-def aggregateUsingIndex[VD2](other: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2]
+class VertexRDD[VD] {
+  // Filter the vertex set but preserves the internal index
+  def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD]
+  // Transform the values without changing the ids (preserves the internal index)
+  def mapValues[VD2](map: VD => VD2): VertexRDD[VD2]
+  def mapValues[VD2](map: (VertexID, VD) => VD2): VertexRDD[VD2]
+  // Remove vertices from this set that appear in the other set
+  def diff(other: VertexRDD[VD]): VertexRDD[VD]
+  // Join operators that take advantage of the internal indexing to accelerate joins (substantially)
+  def leftJoin[VD2, VD3](other: RDD[(VertexID, VD2)])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3]
+  def innerJoin[U, VD2](other: RDD[(VertexID, U)])(f: (VertexID, VD, U) => VD2): VertexRDD[VD2]
+  // Use the index on this RDD to accelerate a `reduceByKey` operation on the input RDD.
+  def aggregateUsingIndex[VD2](other: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2]
+}
 {% endhighlight %}
 
 Notice, for example,  how the `filter` operator returns an `VertexRDD`.  Filter is actually


[15/50] git commit: Improving documentation and identifying potential bug in CC calculation.

Posted by pw...@apache.org.
Improving documentation and identifying potential bug in CC calculation.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/80e4d98d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/80e4d98d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/80e4d98d

Branch: refs/heads/master
Commit: 80e4d98dc656e20dacbd8cdbf92d4912673b42ae
Parents: 8ca9773
Author: Joseph E. Gonzalez <jo...@gmail.com>
Authored: Mon Jan 13 13:40:16 2014 -0800
Committer: Joseph E. Gonzalez <jo...@gmail.com>
Committed: Mon Jan 13 13:40:16 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md                | 33 ++++++++++++---
 .../org/apache/spark/graphx/GraphOps.scala      |  4 +-
 .../spark/graphx/lib/ConnectedComponents.scala  | 44 +++++++++++++-------
 .../graphx/lib/ConnectedComponentsSuite.scala   | 30 +++++++++++++
 4 files changed, 89 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/80e4d98d/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 2697b2d..ed976b8 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -84,7 +84,8 @@ import org.apache.spark.graphx._
 import org.apache.spark.rdd.RDD
 {% endhighlight %}
 
-If you are not using the Spark shell you will also need a Spark context.
+If you are not using the Spark shell you will also need a `SparkContext`.  To learn more about
+getting started with Spark refer to the [Spark Quick Start Guide](quick-start.html).
 
 # The Property Graph
 <a name="property_graph"></a>
@@ -190,7 +191,7 @@ and `graph.edges` members respectively.
 {% highlight scala %}
 val graph: Graph[(String, String), String] // Constructed from above
 // Count all users which are postdocs
-graph.vertices.filter { case (id, (name, pos)) => pos == "postdoc"}.count
+graph.vertices.filter { case (id, (name, pos)) => pos == "postdoc" }.count
 // Count all the edges where src > dst
 graph.edges.filter(e => e.srcId > e.dstId).count
 {% endhighlight %}
@@ -258,8 +259,10 @@ val graph: Graph[(String, String), String]
 val indDegrees: VertexRDD[Int] = graph.inDegrees
 {% endhighlight %}
 
-The reason for differentiating between core graph operations and GraphOps is to be able to support
-various graph representations in the future.
+The reason for differentiating between core graph operations and [`GraphOps`][GraphOps] is to be
+able to support different graph representations in the future.  Each graph representation must
+provide implementations of the core operations and reuse many of the useful operations defined in
+[`GraphOps`][GraphOps].
 
 ## Property Operators
 
@@ -334,14 +337,32 @@ interest or eliminate broken links. For example in the following code we remove
 [Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexID,VD)⇒Boolean):Graph[VD,ED]
 
 {% highlight scala %}
-val users: RDD[(VertexId, (String, String))]
-val edges: RDD[Edge[String]]
+// Create an RDD for the vertices
+val users: RDD[(VertexID, (String, String))] =
+  sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
+                       (5L, ("franklin", "prof")), (2L, ("istoica", "prof")),
+                       (4L, ("peter", "student"))))
+// Create an RDD for edges
+val relationships: RDD[Edge[String]] =
+  sc.parallelize(Array(Edge(3L, 7L, "collab"),    Edge(5L, 3L, "advisor"),
+                       Edge(2L, 5L, "colleague"), Edge(5L, 7L, "pi"),
+                       Edge(4L, 0L, "student"),   Edge(5L, 0L, "colleague")))
 // Define a default user in case there are relationship with missing user
 val defaultUser = ("John Doe", "Missing")
 // Build the initial Graph
 val graph = Graph(users, relationships, defaultUser)
+// Notice that there is a user 0 (for which we have no information) connecting users
+// 4 (peter) and 5 (franklin).
+graph.triplets.map(
+    triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1
+  ).collect.foreach(println(_))
 // Remove missing vertices as well as the edges to connected to them
 val validGraph = graph.subgraph(vpred = (id, attr) => attr._2 != "Missing")
+// The valid subgraph will disconnect users 4 and 5 by removing user 0
+validGraph.vertices.collect.foreach(println(_))
+validGraph.triplets.map(
+    triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1
+  ).collect.foreach(println(_))
 {% endhighlight %}
 
 > Note in the above example only the vertex predicate is provided.  The `subgraph` operator defaults

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/80e4d98d/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 2b3b95e..a0a40e2 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -325,8 +325,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) {
    *
    * @see [[org.apache.spark.graphx.lib.ConnectedComponents]]
    */
-  def connectedComponents(): Graph[VertexID, ED] = {
-    ConnectedComponents.run(graph)
+  def connectedComponents(undirected: Boolean = true): Graph[VertexID, ED] = {
+    ConnectedComponents.run(graph, undirected)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/80e4d98d/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
index 4a83e2d..d078d2a 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
@@ -14,26 +14,42 @@ object ConnectedComponents {
    * @tparam ED the edge attribute type (preserved in the computation)
    *
    * @param graph the graph for which to compute the connected components
+   * @param undirected compute reachability ignoring edge direction.
    *
    * @return a graph with vertex attributes containing the smallest vertex in each
    *         connected component
    */
-  def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = {
+  def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], undirected: Boolean = true):
+    Graph[VertexID, ED] = {
     val ccGraph = graph.mapVertices { case (vid, _) => vid }
-
-    def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
-      if (edge.srcAttr < edge.dstAttr) {
-        Iterator((edge.dstId, edge.srcAttr))
-      } else if (edge.srcAttr > edge.dstAttr) {
-        Iterator((edge.srcId, edge.dstAttr))
-      } else {
-        Iterator.empty
+    if (undirected) {
+      def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
+        if (edge.srcAttr < edge.dstAttr) {
+          Iterator((edge.dstId, edge.srcAttr))
+        } else if (edge.srcAttr > edge.dstAttr) {
+          Iterator((edge.srcId, edge.dstAttr))
+        } else {
+          Iterator.empty
+        }
+      }
+      val initialMessage = Long.MaxValue
+      Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Both)(
+        vprog = (id, attr, msg) => math.min(attr, msg),
+        sendMsg = sendMessage,
+        mergeMsg = (a, b) => math.min(a, b))
+    } else {
+      def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
+        if (edge.srcAttr < edge.dstAttr) {
+          Iterator((edge.dstId, edge.srcAttr))
+        } else {
+          Iterator.empty
+        }
       }
+      val initialMessage = Long.MaxValue
+      Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Out)(
+        vprog = (id, attr, msg) => math.min(attr, msg),
+        sendMsg = sendMessage,
+        mergeMsg = (a, b) => math.min(a, b))
     }
-    val initialMessage = Long.MaxValue
-    Pregel(ccGraph, initialMessage)(
-      vprog = (id, attr, msg) => math.min(attr, msg),
-      sendMsg = sendMessage,
-      mergeMsg = (a, b) => math.min(a, b))
   } // end of connectedComponents
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/80e4d98d/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
----------------------------------------------------------------------
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
index 66612b3..86da8f1 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
@@ -80,4 +80,34 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext {
     }
   } // end of reverse chain connected components
 
+  test("Connected Components on a Toy Connected Graph") {
+    withSpark { sc =>
+      // Create an RDD for the vertices
+      val users: RDD[(VertexID, (String, String))] =
+        sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
+                       (5L, ("franklin", "prof")), (2L, ("istoica", "prof")),
+                       (4L, ("peter", "student"))))
+      // Create an RDD for edges
+      val relationships: RDD[Edge[String]] =
+        sc.parallelize(Array(Edge(3L, 7L, "collab"),    Edge(5L, 3L, "advisor"),
+                       Edge(2L, 5L, "colleague"), Edge(5L, 7L, "pi"),
+                       Edge(4L, 0L, "student"),   Edge(5L, 0L, "colleague")))
+      // Edges are:
+      //   2 ---> 5 ---> 3
+      //          | \
+      //          V   \|
+      //   4 ---> 0    7
+      //
+      // Define a default user in case there are relationship with missing user
+      val defaultUser = ("John Doe", "Missing")
+      // Build the initial Graph
+      val graph = Graph(users, relationships, defaultUser)
+      val ccGraph = graph.connectedComponents(undirected = true)
+      val vertices = ccGraph.vertices.collect
+      for ( (id, cc) <- vertices ) {
+        assert(cc == 0)
+      }
+    }
+  } // end of toy connected components
+
 }


[40/50] git commit: Improve scaladoc links

Posted by pw...@apache.org.
Improve scaladoc links


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c28e5a08
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c28e5a08
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c28e5a08

Branch: refs/heads/master
Commit: c28e5a08eea4f295676ea591c877c648f9f29847
Parents: 59e4384
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 21:11:39 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 21:11:39 2014 -0800

----------------------------------------------------------------------
 .../src/main/scala/org/apache/spark/graphx/EdgeRDD.scala  |  2 +-
 .../src/main/scala/org/apache/spark/graphx/GraphOps.scala | 10 +++++-----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c28e5a08/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
index 0269ed3..c0a23d1 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
@@ -9,7 +9,7 @@ import org.apache.spark.storage.StorageLevel
 
 /**
  * `EdgeRDD[ED]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each partition
- * for performance. It is constructed using [[impl.EdgePartitionBuilder]].
+ * for performance.
  */
 class EdgeRDD[@specialized ED: ClassTag](
     val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])])

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c28e5a08/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 233adb9..f10e63f 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -254,7 +254,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
    * Run a dynamic version of PageRank returning a graph with vertex attributes containing the
    * PageRank and edge attributes containing the normalized edge weight.
    *
-   * @see [[org.apache.spark.graphx.lib.PageRank]], method `runUntilConvergence`.
+   * @see [[org.apache.spark.graphx.lib.PageRank$#runUntilConvergence]]
    */
   def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = {
     PageRank.runUntilConvergence(graph, tol, resetProb)
@@ -264,7 +264,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
    * Run PageRank for a fixed number of iterations returning a graph with vertex attributes
    * containing the PageRank and edge attributes the normalized edge weight.
    *
-   * @see [[org.apache.spark.graphx.lib.PageRank]], method `run`.
+   * @see [[org.apache.spark.graphx.lib.PageRank$#run]]
    */
   def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = {
     PageRank.run(graph, numIter, resetProb)
@@ -274,7 +274,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
    * Compute the connected component membership of each vertex and return a graph with the vertex
    * value containing the lowest vertex id in the connected component containing that vertex.
    *
-   * @see [[org.apache.spark.graphx.lib.ConnectedComponents]]
+   * @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]]
    */
   def connectedComponents(): Graph[VertexID, ED] = {
     ConnectedComponents.run(graph)
@@ -283,7 +283,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
   /**
    * Compute the number of triangles passing through each vertex.
    *
-   * @see [[org.apache.spark.graphx.lib.TriangleCount]]
+   * @see [[org.apache.spark.graphx.lib.TriangleCount$#run]]
    */
   def triangleCount(): Graph[Int, ED] = {
     TriangleCount.run(graph)
@@ -293,7 +293,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
    * Compute the strongly connected component (SCC) of each vertex and return a graph with the
    * vertex value containing the lowest vertex id in the SCC containing that vertex.
    *
-   * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents]]
+   * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents$#run]]
    */
   def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = {
     StronglyConnectedComponents.run(graph, numIter)


[05/50] git commit: Add connected components example to doc

Posted by pw...@apache.org.
Add connected components example to doc


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/7a4bb863
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/7a4bb863
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/7a4bb863

Branch: refs/heads/master
Commit: 7a4bb863c7c11e22332763081793e4989af8c526
Parents: 5e35d39
Author: Ankur Dave <an...@gmail.com>
Authored: Sun Jan 12 16:58:18 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Sun Jan 12 16:58:18 2014 -0800

----------------------------------------------------------------------
 docs/graphx-programming-guide.md | 20 +++++++++++++++++++-
 graphx/data/followers.txt        |  6 +-----
 graphx/data/users.txt            |  2 +-
 3 files changed, 21 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/7a4bb863/docs/graphx-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 52668b0..22feccb 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -475,6 +475,7 @@ GraphX includes a set of graph algorithms in to simplify analytics. The algorith
 [Algorithms]: api/graphx/index.html#org.apache.spark.graphx.lib.Algorithms
 
 ## PageRank
+<a name="pagerank"></a>
 
 PageRank measures the importance of each vertex in a graph, assuming an edge from *u* to *v* represents an endorsement of *v*'s importance by *u*. For example, if a Twitter user is followed by many others, the user will be ranked highly.
 
@@ -503,9 +504,26 @@ val ranksByUsername = users.leftOuterJoin(ranks).map {
 println(ranksByUsername.collect().mkString("\n"))
 {% endhighlight %}
 
-
 ## Connected Components
 
+The connected components algorithm labels each connected component of the graph with the ID of its lowest-numbered vertex. For example, in a social network, connected components can approximate clusters. We can compute the connected components of the example social network dataset from the [PageRank section](#pagerank) as follows:
+
+{% highlight scala %}
+// Load the implicit conversion and graph as in the PageRank example
+import org.apache.spark.graphx.lib._
+val users = ...
+val followers = ...
+val graph = Graph(users, followers)
+// Find the connected components
+val cc = graph.connectedComponents().vertices
+// Join the connected components with the usernames
+val ccByUsername = graph.vertices.innerJoin(cc) { (id, username, cc) =>
+  (username, cc)
+}
+// Print the result
+println(ccByUsername.collect().mkString("\n"))
+{% endhighlight %}
+
 ## Shortest Path
 
 ## Triangle Counting

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/7a4bb863/graphx/data/followers.txt
----------------------------------------------------------------------
diff --git a/graphx/data/followers.txt b/graphx/data/followers.txt
index 0f46d80..7bb8e90 100644
--- a/graphx/data/followers.txt
+++ b/graphx/data/followers.txt
@@ -1,10 +1,6 @@
 2 1
-3 1
 4 1
-6 1
-3 2
-6 2
-7 2
+1 2
 6 3
 7 3
 7 6

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/7a4bb863/graphx/data/users.txt
----------------------------------------------------------------------
diff --git a/graphx/data/users.txt b/graphx/data/users.txt
index ce3d06c..26e3b3b 100644
--- a/graphx/data/users.txt
+++ b/graphx/data/users.txt
@@ -1,5 +1,5 @@
 1 BarackObama
-2 ericschmidt
+2 ladygaga
 3 jeresig
 4 justinbieber
 6 matei_zaharia


[14/50] git commit: Improve EdgeRDD scaladoc

Posted by pw...@apache.org.
Improve EdgeRDD scaladoc


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/9fe88627
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/9fe88627
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/9fe88627

Branch: refs/heads/master
Commit: 9fe88627b5716c6893a6d034fefcf530a6c470fa
Parents: ea69cff
Author: Ankur Dave <an...@gmail.com>
Authored: Mon Jan 13 13:16:41 2014 -0800
Committer: Ankur Dave <an...@gmail.com>
Committed: Mon Jan 13 13:16:41 2014 -0800

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/graphx/EdgeRDD.scala   | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9fe88627/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
----------------------------------------------------------------------
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
index 05d3dbe..0269ed3 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
@@ -49,7 +49,7 @@ class EdgeRDD[@specialized ED: ClassTag](
     this
   }
 
-  def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2])
+  private[graphx] def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2])
     : EdgeRDD[ED2] = {
     new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter =>
       val (pid, ep) = iter.next()
@@ -57,6 +57,15 @@ class EdgeRDD[@specialized ED: ClassTag](
     }, preservesPartitioning = true))
   }
 
+  /**
+   * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same
+   * [[PartitionStrategy]].
+   *
+   * @param other the EdgeRDD to join with
+   * @param f the join function applied to corresponding values of `this` and `other`
+   * @return a new EdgeRDD containing only edges that appear in both `this` and `other`, with values
+   * supplied by `f`
+   */
   def innerJoin[ED2: ClassTag, ED3: ClassTag]
       (other: EdgeRDD[ED2])
       (f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3] = {
@@ -70,7 +79,7 @@ class EdgeRDD[@specialized ED: ClassTag](
     })
   }
 
-  def collectVertexIDs(): RDD[VertexID] = {
+  private[graphx] def collectVertexIDs(): RDD[VertexID] = {
     partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) }
   }
 }