Skip to content

Commit

Permalink
[SPARK-1931] Reconstruct routing tables in Graph.partitionBy
Browse files Browse the repository at this point in the history
905173d introduced a bug in partitionBy where, after repartitioning the edges, it reuses the VertexRDD without updating the routing tables to reflect the new edge layout. Subsequent accesses of the triplets contain nulls for many vertex properties.

This commit adds a test for this bug and fixes it by introducing `VertexRDD#withEdges` and calling it in `partitionBy`.

Author: Ankur Dave <ankurdave@gmail.com>

Closes #885 from ankurdave/SPARK-1931 and squashes the following commits:

3930cdd [Ankur Dave] Note how to set up VertexRDD for efficient joins
9bdbaa4 [Ankur Dave] [SPARK-1931] Reconstruct routing tables in Graph.partitionBy
  • Loading branch information
ankurdave authored and rxin committed May 26, 2014
1 parent cb7fe50 commit 56c771c
Show file tree
Hide file tree
Showing 3 changed files with 31 additions and 4 deletions.
12 changes: 12 additions & 0 deletions graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,18 @@ class VertexRDD[@specialized VD: ClassTag](
def reverseRoutingTables(): VertexRDD[VD] =
this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse))

/** Prepares this VertexRDD for efficient joins with the given EdgeRDD. */
def withEdges(edges: EdgeRDD[_, _]): VertexRDD[VD] = {
val routingTables = VertexRDD.createRoutingTables(edges, this.partitioner.get)
val vertexPartitions = partitionsRDD.zipPartitions(routingTables, true) {
(partIter, routingTableIter) =>
val routingTable =
if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty
partIter.map(_.withRoutingTable(routingTable))
}
new VertexRDD(vertexPartitions)
}

/** Generates an RDD of vertex attributes suitable for shipping to the edge partitions. */
private[graphx] def shipVertexAttributes(
shipSrc: Boolean, shipDst: Boolean): RDD[(PartitionID, VertexAttributeBlock[VD])] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,8 +88,8 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
}
val edgePartition = builder.toEdgePartition
Iterator((pid, edgePartition))
}, preservesPartitioning = true))
GraphImpl.fromExistingRDDs(vertices, newEdges)
}, preservesPartitioning = true)).cache()
GraphImpl.fromExistingRDDs(vertices.withEdges(newEdges), newEdges)
}

override def reverse: Graph[VD, ED] = {
Expand Down Expand Up @@ -277,7 +277,11 @@ object GraphImpl {
GraphImpl(vertexRDD, edgeRDD)
}

/** Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. */
/**
* Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. The
* VertexRDD must already be set up for efficient joins with the EdgeRDD by calling
* `VertexRDD.withEdges` or an appropriate VertexRDD constructor.
*/
def apply[VD: ClassTag, ED: ClassTag](
vertices: VertexRDD[VD],
edges: EdgeRDD[ED, _]): GraphImpl[VD, ED] = {
Expand All @@ -290,7 +294,8 @@ object GraphImpl {

/**
* Create a graph from a VertexRDD and an EdgeRDD with the same replicated vertex type as the
* vertices.
* vertices. The VertexRDD must already be set up for efficient joins with the EdgeRDD by calling
* `VertexRDD.withEdges` or an appropriate VertexRDD constructor.
*/
def fromExistingRDDs[VD: ClassTag, ED: ClassTag](
vertices: VertexRDD[VD],
Expand Down
10 changes: 10 additions & 0 deletions graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,16 @@ class GraphSuite extends FunSuite with LocalSparkContext {
Iterator((part.srcIds ++ part.dstIds).toSet)
}.collect
assert(verts.exists(id => partitionSetsUnpartitioned.count(_.contains(id)) > bound))

// Forming triplets view
val g = Graph(
sc.parallelize(List((0L, "a"), (1L, "b"), (2L, "c"))),
sc.parallelize(List(Edge(0L, 1L, 1), Edge(0L, 2L, 1)), 2))
assert(g.triplets.collect.map(_.toTuple).toSet ===
Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1)))
val gPart = g.partitionBy(EdgePartition2D)
assert(gPart.triplets.collect.map(_.toTuple).toSet ===
Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1)))
}
}

Expand Down

1 comment on commit 56c771c

@npanj
Copy link

@npanj npanj commented on 56c771c May 27, 2014

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi Ankur - I am seeing something strange with outerJoinVertices(and triangle count that relies on this api), and I was wondering if that could be related to this bug that you are patching?(i.e. likely fixed by your patch?).

Here is what I am doing:

  1. Created a Graph with multiple partitions i.e created a graph with minEdgePartitions, where minEdgePartitions >=1; and use partitionBy(PartitionStrategy.RandomVertexCut) on generated graph. Note: vertex attribute type is Int in this case
  2. next I am building neighborhood ids by calling collectNeighborIds i.e. returned vertex attribute type is Array[VertexId] ; VertexRDD[Array[VertexId]]
  3. finally join vertex ids from 2 to graph (generated in step 1) via outerJoinVertices
  4. Create a subgraph on joined graph from 3 where I only keep the edges with ed.srcAttr != -1 && ed.dstAttr != -1 i.e. filter out null attr vertices
  5. Finally checked the number edges left in subgraph from step4

I ran this program in a loop where minEdgePartitions is changed in each iteration. When minEdgePartitions == 1 I see correct number of edges. When minEdgePartitions == 2 result is ~1/2 number of edges; when minEdgePartitions == 3 result is ~1/3 number of edges and so on

It seems that outerJoinVertices is returning srcAttr/dstAtt = nulll for many attributes; and from numbers it seems that it might be returning null for vertices residing on other partitions ?

Environment : I am using RC5; and 22 executers.

BUT I get correct number of edges in each iteration when I repeated my experiment by keeping the vertex attribute type Int in step 2 (i.e. just kept the number of vertices instead of array of vertices), which is same as the type vertex attribute in graph before join.

Please sign in to comment.